HDFS-5191. Revisit zero-copy API in FSDataInputStream to make it more intuitive (Contributed by Colin Patrick McCabe)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1526020 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
552e081a07
commit
9a361c5821
|
@ -0,0 +1,113 @@
|
||||||
|
/**
|
||||||
|
* 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.io.InputStream;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.io.ByteBufferPool;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public final class ByteBufferUtil {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Determine if a stream can do a byte buffer read via read(ByteBuffer buf)
|
||||||
|
*/
|
||||||
|
private static boolean streamHasByteBufferRead(InputStream stream) {
|
||||||
|
if (!(stream instanceof ByteBufferReadable)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!(stream instanceof FSDataInputStream)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
return ((FSDataInputStream)stream).getWrappedStream()
|
||||||
|
instanceof ByteBufferReadable;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Perform a fallback read.
|
||||||
|
*/
|
||||||
|
public static ByteBuffer fallbackRead(
|
||||||
|
InputStream stream, ByteBufferPool bufferPool, int maxLength)
|
||||||
|
throws IOException {
|
||||||
|
if (bufferPool == null) {
|
||||||
|
throw new UnsupportedOperationException("zero-copy reads " +
|
||||||
|
"were not available, and you did not provide a fallback " +
|
||||||
|
"ByteBufferPool.");
|
||||||
|
}
|
||||||
|
boolean useDirect = streamHasByteBufferRead(stream);
|
||||||
|
ByteBuffer buffer = bufferPool.getBuffer(useDirect, maxLength);
|
||||||
|
if (buffer == null) {
|
||||||
|
throw new UnsupportedOperationException("zero-copy reads " +
|
||||||
|
"were not available, and the ByteBufferPool did not provide " +
|
||||||
|
"us with " + (useDirect ? "a direct" : "an indirect") +
|
||||||
|
"buffer.");
|
||||||
|
}
|
||||||
|
Preconditions.checkState(buffer.capacity() > 0);
|
||||||
|
Preconditions.checkState(buffer.isDirect() == useDirect);
|
||||||
|
maxLength = Math.min(maxLength, buffer.capacity());
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
if (useDirect) {
|
||||||
|
buffer.clear();
|
||||||
|
buffer.limit(maxLength);
|
||||||
|
ByteBufferReadable readable = (ByteBufferReadable)stream;
|
||||||
|
int totalRead = 0;
|
||||||
|
while (true) {
|
||||||
|
if (totalRead >= maxLength) {
|
||||||
|
success = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
int nRead = readable.read(buffer);
|
||||||
|
if (nRead < 0) {
|
||||||
|
if (totalRead > 0) {
|
||||||
|
success = true;
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
totalRead += nRead;
|
||||||
|
}
|
||||||
|
buffer.flip();
|
||||||
|
} else {
|
||||||
|
buffer.clear();
|
||||||
|
int nRead = stream.read(buffer.array(),
|
||||||
|
buffer.arrayOffset(), maxLength);
|
||||||
|
if (nRead >= 0) {
|
||||||
|
buffer.limit(nRead);
|
||||||
|
success = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
if (!success) {
|
||||||
|
// If we got an error while reading, or if we are at EOF, we
|
||||||
|
// don't need the buffer any more. We can give it back to the
|
||||||
|
// bufferPool.
|
||||||
|
bufferPool.putBuffer(buffer);
|
||||||
|
buffer = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,4 +1,5 @@
|
||||||
/**
|
/**
|
||||||
|
*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -19,9 +20,13 @@ package org.apache.hadoop.fs;
|
||||||
|
|
||||||
import java.io.*;
|
import java.io.*;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.io.ByteBufferPool;
|
||||||
|
import org.apache.hadoop.fs.ByteBufferUtil;
|
||||||
|
import org.apache.hadoop.util.IdentityHashStore;
|
||||||
|
|
||||||
/** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
|
/** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
|
||||||
* and buffers input through a {@link BufferedInputStream}. */
|
* and buffers input through a {@link BufferedInputStream}. */
|
||||||
|
@ -30,7 +35,15 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
public class FSDataInputStream extends DataInputStream
|
public class FSDataInputStream extends DataInputStream
|
||||||
implements Seekable, PositionedReadable, Closeable,
|
implements Seekable, PositionedReadable, Closeable,
|
||||||
ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead,
|
ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead,
|
||||||
SupportsZeroCopy {
|
HasEnhancedByteBufferAccess {
|
||||||
|
/**
|
||||||
|
* Map ByteBuffers that we have handed out to readers to ByteBufferPool
|
||||||
|
* objects
|
||||||
|
*/
|
||||||
|
private final IdentityHashStore<ByteBuffer, ByteBufferPool>
|
||||||
|
extendedReadBuffers
|
||||||
|
= new IdentityHashStore<ByteBuffer, ByteBufferPool>(0);
|
||||||
|
|
||||||
public FSDataInputStream(InputStream in)
|
public FSDataInputStream(InputStream in)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(in);
|
super(in);
|
||||||
|
@ -169,13 +182,43 @@ public class FSDataInputStream extends DataInputStream
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ZeroCopyCursor createZeroCopyCursor()
|
public ByteBuffer read(ByteBufferPool bufferPool, int maxLength,
|
||||||
throws IOException, ZeroCopyUnavailableException {
|
EnumSet<ReadOption> opts)
|
||||||
|
throws IOException, UnsupportedOperationException {
|
||||||
try {
|
try {
|
||||||
return ((SupportsZeroCopy)in).createZeroCopyCursor();
|
return ((HasEnhancedByteBufferAccess)in).read(bufferPool,
|
||||||
|
maxLength, opts);
|
||||||
}
|
}
|
||||||
catch (ClassCastException e) {
|
catch (ClassCastException e) {
|
||||||
throw new ZeroCopyUnavailableException(e);
|
ByteBuffer buffer = ByteBufferUtil.
|
||||||
|
fallbackRead(this, bufferPool, maxLength);
|
||||||
|
if (buffer != null) {
|
||||||
|
extendedReadBuffers.put(buffer, bufferPool);
|
||||||
|
}
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final EnumSet<ReadOption> EMPTY_READ_OPTIONS_SET =
|
||||||
|
EnumSet.noneOf(ReadOption.class);
|
||||||
|
|
||||||
|
final public ByteBuffer read(ByteBufferPool bufferPool, int maxLength)
|
||||||
|
throws IOException, UnsupportedOperationException {
|
||||||
|
return read(bufferPool, maxLength, EMPTY_READ_OPTIONS_SET);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void releaseBuffer(ByteBuffer buffer) {
|
||||||
|
try {
|
||||||
|
((HasEnhancedByteBufferAccess)in).releaseBuffer(buffer);
|
||||||
|
}
|
||||||
|
catch (ClassCastException e) {
|
||||||
|
ByteBufferPool bufferPool = extendedReadBuffers.remove( buffer);
|
||||||
|
if (bufferPool == null) {
|
||||||
|
throw new IllegalArgumentException("tried to release a buffer " +
|
||||||
|
"that was not created by this stream.");
|
||||||
|
}
|
||||||
|
bufferPool.putBuffer(buffer);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.fs.ZeroCopyUnavailableException;
|
||||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public abstract class FSInputStream extends InputStream
|
public abstract class FSInputStream extends InputStream
|
||||||
implements Seekable, PositionedReadable, SupportsZeroCopy {
|
implements Seekable, PositionedReadable {
|
||||||
/**
|
/**
|
||||||
* Seek to the given offset from the start of the file.
|
* Seek to the given offset from the start of the file.
|
||||||
* The next read() will be from that location. Can't
|
* The next read() will be from that location. Can't
|
||||||
|
@ -88,11 +88,4 @@ public abstract class FSInputStream extends InputStream
|
||||||
throws IOException {
|
throws IOException {
|
||||||
readFully(position, buffer, 0, buffer.length);
|
readFully(position, buffer, 0, buffer.length);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public ZeroCopyCursor createZeroCopyCursor()
|
|
||||||
throws IOException, ZeroCopyUnavailableException {
|
|
||||||
throw new ZeroCopyUnavailableException("zero copy is not implemented " +
|
|
||||||
"for this filesystem type.");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,79 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.io.ByteBufferPool;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* FSDataInputStreams implement this interface to provide enhanced
|
||||||
|
* byte buffer access. Usually this takes the form of mmap support.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public interface HasEnhancedByteBufferAccess {
|
||||||
|
/**
|
||||||
|
* Get a ByteBuffer containing file data.
|
||||||
|
*
|
||||||
|
* This ByteBuffer may come from the stream itself, via a call like mmap,
|
||||||
|
* or it may come from the ByteBufferFactory which is passed in as an
|
||||||
|
* argument.
|
||||||
|
*
|
||||||
|
* @param factory
|
||||||
|
* If this is non-null, it will be used to create a fallback
|
||||||
|
* ByteBuffer when the stream itself cannot create one.
|
||||||
|
* @param maxLength
|
||||||
|
* The maximum length of buffer to return. We may return a buffer
|
||||||
|
* which is shorter than this.
|
||||||
|
* @param opts
|
||||||
|
* Options to use when reading.
|
||||||
|
*
|
||||||
|
* @return
|
||||||
|
* We will return null on EOF (and only on EOF).
|
||||||
|
* Otherwise, we will return a direct ByteBuffer containing at
|
||||||
|
* least one byte. You must free this ByteBuffer when you are
|
||||||
|
* done with it by calling releaseBuffer on it.
|
||||||
|
* The buffer will continue to be readable until it is released
|
||||||
|
* in this manner. However, the input stream's close method may
|
||||||
|
* warn about unclosed buffers.
|
||||||
|
* @throws
|
||||||
|
* IOException: if there was an error reading.
|
||||||
|
* UnsupportedOperationException: if factory was null, and we
|
||||||
|
* needed an external byte buffer. UnsupportedOperationException
|
||||||
|
* will never be thrown unless the factory argument is null.
|
||||||
|
*/
|
||||||
|
public ByteBuffer read(ByteBufferPool factory, int maxLength,
|
||||||
|
EnumSet<ReadOption> opts)
|
||||||
|
throws IOException, UnsupportedOperationException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Release a ByteBuffer which was created by the enhanced ByteBuffer read
|
||||||
|
* function. You must not continue using the ByteBuffer after calling this
|
||||||
|
* function.
|
||||||
|
*
|
||||||
|
* @param buffer
|
||||||
|
* The ByteBuffer to release.
|
||||||
|
*/
|
||||||
|
public void releaseBuffer(ByteBuffer buffer);
|
||||||
|
}
|
|
@ -15,30 +15,20 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.fs;
|
package org.apache.hadoop.fs;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Supports zero-copy reads.
|
* Options that can be used when reading from a FileSystem.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Public
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Stable
|
||||||
public interface SupportsZeroCopy {
|
public enum ReadOption {
|
||||||
/**
|
/**
|
||||||
* Get a zero-copy cursor to use for zero-copy reads.
|
* Skip checksums when reading. This option may be useful when reading a file
|
||||||
*
|
* format that has built-in checksums, or for testing purposes.
|
||||||
* @throws IOException
|
|
||||||
* If there was an error creating the ZeroCopyCursor
|
|
||||||
* @throws UnsupportedOperationException
|
|
||||||
* If this stream does not support zero-copy reads.
|
|
||||||
* This is used, for example, when one stream wraps another
|
|
||||||
* which may or may not support ZCR.
|
|
||||||
*/
|
*/
|
||||||
public ZeroCopyCursor createZeroCopyCursor()
|
SKIP_CHECKSUMS,
|
||||||
throws IOException, ZeroCopyUnavailableException;
|
}
|
||||||
}
|
|
|
@ -1,111 +0,0 @@
|
||||||
/**
|
|
||||||
* 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.Closeable;
|
|
||||||
import java.io.EOFException;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A ZeroCopyCursor allows you to make zero-copy reads.
|
|
||||||
*
|
|
||||||
* Cursors should be closed when they are no longer needed.
|
|
||||||
*
|
|
||||||
* Example:
|
|
||||||
* FSDataInputStream fis = fs.open("/file");
|
|
||||||
* ZeroCopyCursor cursor = fis.createZeroCopyCursor();
|
|
||||||
* try {
|
|
||||||
* cursor.read(128);
|
|
||||||
* ByteBuffer data = cursor.getData();
|
|
||||||
* processData(data);
|
|
||||||
* } finally {
|
|
||||||
* cursor.close();
|
|
||||||
* }
|
|
||||||
*/
|
|
||||||
public interface ZeroCopyCursor extends Closeable {
|
|
||||||
/**
|
|
||||||
* Set the fallback buffer used for this zero copy cursor.
|
|
||||||
* The fallback buffer is used when a true zero-copy read is impossible.
|
|
||||||
* If there is no fallback buffer, UnsupportedOperationException is thrown
|
|
||||||
* when a true zero-copy read cannot be done.
|
|
||||||
*
|
|
||||||
* @param fallbackBuffer The fallback buffer to set, or null for none.
|
|
||||||
*/
|
|
||||||
public void setFallbackBuffer(ByteBuffer fallbackBuffer);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the fallback buffer in use, or null if there is none.
|
|
||||||
*/
|
|
||||||
public ByteBuffer getFallbackBuffer();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param skipChecksums Whether we should skip checksumming with this
|
|
||||||
* zero copy cursor.
|
|
||||||
*/
|
|
||||||
public void setSkipChecksums(boolean skipChecksums);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return Whether we should skip checksumming with this
|
|
||||||
* zero copy cursor.
|
|
||||||
*/
|
|
||||||
public boolean getSkipChecksums();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param allowShortReads Whether we should allow short reads.
|
|
||||||
*/
|
|
||||||
public void setAllowShortReads(boolean allowShortReads);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return Whether we should allow short reads.
|
|
||||||
*/
|
|
||||||
public boolean getAllowShortReads();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Perform a zero-copy read.
|
|
||||||
*
|
|
||||||
* @param toRead The minimum number of bytes to read.
|
|
||||||
* Must not be negative. If we hit EOF before
|
|
||||||
* reading this many bytes, we will either throw
|
|
||||||
* EOFException (if allowShortReads = false), or
|
|
||||||
* return a short read (if allowShortReads = true).
|
|
||||||
* A short read could be as short as 0 bytes.
|
|
||||||
* @throws UnsupportedOperationException
|
|
||||||
* If a true zero-copy read cannot be done, and no fallback
|
|
||||||
* buffer was set.
|
|
||||||
* @throws EOFException
|
|
||||||
* If allowShortReads = false, and we can't read all the bytes
|
|
||||||
* that were requested. This will never be thrown if
|
|
||||||
* allowShortReads = true.
|
|
||||||
* @throws IOException
|
|
||||||
* If there was an error while reading the data.
|
|
||||||
*/
|
|
||||||
public void read(int toRead)
|
|
||||||
throws UnsupportedOperationException, EOFException, IOException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the current data buffer.
|
|
||||||
*
|
|
||||||
* This buffer will remain valid until either this cursor is closed, or we
|
|
||||||
* call read() again on this same cursor. You can find the amount of data
|
|
||||||
* that was read previously by calling ByteBuffer#remaining.
|
|
||||||
*
|
|
||||||
* @return The current data buffer.
|
|
||||||
*/
|
|
||||||
public ByteBuffer getData();
|
|
||||||
}
|
|
|
@ -0,0 +1,48 @@
|
||||||
|
/**
|
||||||
|
* 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.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
@InterfaceStability.Stable
|
||||||
|
public interface ByteBufferPool {
|
||||||
|
/**
|
||||||
|
* Get a new direct ByteBuffer. The pool can provide this from
|
||||||
|
* removing a buffer from its internal cache, or by allocating a
|
||||||
|
* new buffer.
|
||||||
|
*
|
||||||
|
* @param direct Whether the buffer should be direct.
|
||||||
|
* @param minLength The minimum length the buffer will have.
|
||||||
|
* @return A new ByteBuffer. This ByteBuffer must be direct.
|
||||||
|
* Its capacity can be less than what was requested, but
|
||||||
|
* must be at least 1 byte.
|
||||||
|
*/
|
||||||
|
ByteBuffer getBuffer(boolean direct, int length);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Release a buffer back to the pool.
|
||||||
|
* The pool may choose to put this buffer into its cache.
|
||||||
|
*
|
||||||
|
* @param buffer a direct bytebuffer
|
||||||
|
*/
|
||||||
|
void putBuffer(ByteBuffer buffer);
|
||||||
|
}
|
|
@ -0,0 +1,96 @@
|
||||||
|
/**
|
||||||
|
* 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 com.google.common.collect.ComparisonChain;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.TreeMap;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is a simple ByteBufferPool which just creates ByteBuffers as needed.
|
||||||
|
* It also caches ByteBuffers after they're released. It will always return
|
||||||
|
* the smallest cached buffer with at least the capacity you request.
|
||||||
|
* We don't try to do anything clever here like try to limit the maximum cache
|
||||||
|
* size.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
@InterfaceStability.Stable
|
||||||
|
public final class ElasticByteBufferPool implements ByteBufferPool {
|
||||||
|
private static final class Key implements Comparable<Key> {
|
||||||
|
private final int capacity;
|
||||||
|
private final long insertionTime;
|
||||||
|
|
||||||
|
Key(int capacity, long insertionTime) {
|
||||||
|
this.capacity = capacity;
|
||||||
|
this.insertionTime = insertionTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compareTo(Key other) {
|
||||||
|
return ComparisonChain.start().
|
||||||
|
compare(capacity, other.capacity).
|
||||||
|
compare(insertionTime, other.insertionTime).
|
||||||
|
result();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private final TreeMap<Key, ByteBuffer> buffers =
|
||||||
|
new TreeMap<Key, ByteBuffer>();
|
||||||
|
|
||||||
|
private final TreeMap<Key, ByteBuffer> directBuffers =
|
||||||
|
new TreeMap<Key, ByteBuffer>();
|
||||||
|
|
||||||
|
private final TreeMap<Key, ByteBuffer> getBufferTree(boolean direct) {
|
||||||
|
return direct ? directBuffers : buffers;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized ByteBuffer getBuffer(boolean direct, int length) {
|
||||||
|
TreeMap<Key, ByteBuffer> tree = getBufferTree(direct);
|
||||||
|
Map.Entry<Key, ByteBuffer> entry =
|
||||||
|
tree.ceilingEntry(new Key(length, 0));
|
||||||
|
if (entry == null) {
|
||||||
|
return direct ? ByteBuffer.allocateDirect(length) :
|
||||||
|
ByteBuffer.allocate(length);
|
||||||
|
}
|
||||||
|
tree.remove(entry.getKey());
|
||||||
|
return entry.getValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void putBuffer(ByteBuffer buffer) {
|
||||||
|
TreeMap<Key, ByteBuffer> tree = getBufferTree(buffer.isDirect());
|
||||||
|
while (true) {
|
||||||
|
Key key = new Key(buffer.capacity(), System.nanoTime());
|
||||||
|
if (!tree.containsKey(key)) {
|
||||||
|
tree.put(key, buffer);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// Buffers are indexed by (capacity, time).
|
||||||
|
// If our key is not unique on the first try, we try again, since the
|
||||||
|
// time will be different. Since we use nanoseconds, it's pretty
|
||||||
|
// unlikely that we'll loop even once, unless the system clock has a
|
||||||
|
// poor granularity.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,197 @@
|
||||||
|
/**
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The IdentityHashStore stores (key, value) mappings in an array.
|
||||||
|
* It is similar to java.util.HashTable, but much more lightweight.
|
||||||
|
* Neither inserting nor removing an element ever leads to any garbage
|
||||||
|
* getting created (assuming the array doesn't need to be enlarged).
|
||||||
|
*
|
||||||
|
* Unlike HashTable, it compares keys using
|
||||||
|
* {@link System#identityHashCode(Object)} and the identity operator.
|
||||||
|
* This is useful for types like ByteBuffer which have expensive hashCode
|
||||||
|
* and equals operators.
|
||||||
|
*
|
||||||
|
* We use linear probing to resolve collisions. This avoids the need for
|
||||||
|
* the overhead of linked list data structures. It also means that it is
|
||||||
|
* expensive to attempt to remove an element that isn't there, since we
|
||||||
|
* have to look at the entire array to be sure that it doesn't exist.
|
||||||
|
*
|
||||||
|
* @param <K> The key type to use.
|
||||||
|
* @param <V> THe value type to use.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public final class IdentityHashStore<K, V> {
|
||||||
|
/**
|
||||||
|
* Even elements are keys; odd elements are values.
|
||||||
|
* The array has size 1 + Math.pow(2, capacity).
|
||||||
|
*/
|
||||||
|
private Object buffer[];
|
||||||
|
|
||||||
|
private int numInserted = 0;
|
||||||
|
|
||||||
|
private int capacity;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The default maxCapacity value to use.
|
||||||
|
*/
|
||||||
|
private static final int DEFAULT_MAX_CAPACITY = 2;
|
||||||
|
|
||||||
|
public IdentityHashStore(int capacity) {
|
||||||
|
Preconditions.checkArgument(capacity >= 0);
|
||||||
|
if (capacity == 0) {
|
||||||
|
this.capacity = 0;
|
||||||
|
this.buffer = null;
|
||||||
|
} else {
|
||||||
|
// Round the capacity we need up to a power of 2.
|
||||||
|
realloc((int)Math.pow(2,
|
||||||
|
Math.ceil(Math.log(capacity) / Math.log(2))));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void realloc(int newCapacity) {
|
||||||
|
Preconditions.checkArgument(newCapacity > 0);
|
||||||
|
Object prevBuffer[] = buffer;
|
||||||
|
this.capacity = newCapacity;
|
||||||
|
// Each element takes two array slots -- one for the key,
|
||||||
|
// and another for the value. We also want a load factor
|
||||||
|
// of 0.50. Combine those together and you get 4 * newCapacity.
|
||||||
|
this.buffer = new Object[4 * newCapacity];
|
||||||
|
this.numInserted = 0;
|
||||||
|
if (prevBuffer != null) {
|
||||||
|
for (int i = 0; i < prevBuffer.length; i += 2) {
|
||||||
|
if (prevBuffer[i] != null) {
|
||||||
|
putInternal(prevBuffer[i], prevBuffer[i + 1]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void putInternal(Object k, Object v) {
|
||||||
|
int hash = System.identityHashCode(k);
|
||||||
|
final int numEntries = buffer.length / 2;
|
||||||
|
int index = hash % numEntries;
|
||||||
|
while (true) {
|
||||||
|
if (buffer[2 * index] == null) {
|
||||||
|
buffer[2 * index] = k;
|
||||||
|
buffer[1 + (2 * index)] = v;
|
||||||
|
numInserted++;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
index = (index + 1) % numEntries;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a new (key, value) mapping.
|
||||||
|
*
|
||||||
|
* Inserting a new (key, value) never overwrites a previous one.
|
||||||
|
* In other words, you can insert the same key multiple times and it will
|
||||||
|
* lead to multiple entries.
|
||||||
|
*/
|
||||||
|
public void put(K k, V v) {
|
||||||
|
Preconditions.checkNotNull(k);
|
||||||
|
if (buffer == null) {
|
||||||
|
realloc(DEFAULT_MAX_CAPACITY);
|
||||||
|
} else if (numInserted + 1 > capacity) {
|
||||||
|
realloc(capacity * 2);
|
||||||
|
}
|
||||||
|
putInternal(k, v);
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getElementIndex(K k) {
|
||||||
|
if (buffer == null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
final int numEntries = buffer.length / 2;
|
||||||
|
int hash = System.identityHashCode(k);
|
||||||
|
int index = hash % numEntries;
|
||||||
|
int firstIndex = index;
|
||||||
|
do {
|
||||||
|
if (buffer[2 * index] == k) {
|
||||||
|
return index;
|
||||||
|
}
|
||||||
|
index = (index + 1) % numEntries;
|
||||||
|
} while (index != firstIndex);
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve a value associated with a given key.
|
||||||
|
*/
|
||||||
|
public V get(K k) {
|
||||||
|
int index = getElementIndex(k);
|
||||||
|
if (index < 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return (V)buffer[1 + (2 * index)];
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve a value associated with a given key, and delete the
|
||||||
|
* relevant entry.
|
||||||
|
*/
|
||||||
|
public V remove(K k) {
|
||||||
|
int index = getElementIndex(k);
|
||||||
|
if (index < 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
V val = (V)buffer[1 + (2 * index)];
|
||||||
|
buffer[2 * index] = null;
|
||||||
|
buffer[1 + (2 * index)] = null;
|
||||||
|
numInserted--;
|
||||||
|
return val;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isEmpty() {
|
||||||
|
return numInserted == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int numElements() {
|
||||||
|
return numInserted;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int capacity() {
|
||||||
|
return capacity;
|
||||||
|
}
|
||||||
|
|
||||||
|
public interface Visitor<K, V> {
|
||||||
|
void accept(K k, V v);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Visit all key, value pairs in the IdentityHashStore.
|
||||||
|
*/
|
||||||
|
public void visitAll(Visitor<K, V> visitor) {
|
||||||
|
int length = buffer == null ? 0 : buffer.length;
|
||||||
|
for (int i = 0; i < length; i += 2) {
|
||||||
|
if (buffer[i] != null) {
|
||||||
|
visitor.accept((K)buffer[i], (V)buffer[i + 1]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,159 @@
|
||||||
|
/**
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import junit.framework.Assert;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.util.IdentityHashStore;
|
||||||
|
import org.apache.hadoop.util.IdentityHashStore.Visitor;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class TestIdentityHashStore {
|
||||||
|
private static final Log LOG = LogFactory.getLog(TestIdentityHashStore.class.getName());
|
||||||
|
|
||||||
|
private static class Key {
|
||||||
|
private final String name;
|
||||||
|
|
||||||
|
Key(String name) {
|
||||||
|
this.name = name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
throw new RuntimeException("should not be used!");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (!(o instanceof Key)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Key other = (Key)o;
|
||||||
|
return name.equals(other.name);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=60000)
|
||||||
|
public void testStartingWithZeroCapacity() {
|
||||||
|
IdentityHashStore<Key, Integer> store =
|
||||||
|
new IdentityHashStore<Key, Integer>(0);
|
||||||
|
store.visitAll(new Visitor<Key, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void accept(Key k, Integer v) {
|
||||||
|
Assert.fail("found key " + k + " in empty IdentityHashStore.");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
Assert.assertTrue(store.isEmpty());
|
||||||
|
final Key key1 = new Key("key1");
|
||||||
|
Integer value1 = new Integer(100);
|
||||||
|
store.put(key1, value1);
|
||||||
|
Assert.assertTrue(!store.isEmpty());
|
||||||
|
Assert.assertEquals(value1, store.get(key1));
|
||||||
|
store.visitAll(new Visitor<Key, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void accept(Key k, Integer v) {
|
||||||
|
Assert.assertEquals(key1, k);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
Assert.assertEquals(value1, store.remove(key1));
|
||||||
|
Assert.assertTrue(store.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=60000)
|
||||||
|
public void testDuplicateInserts() {
|
||||||
|
IdentityHashStore<Key, Integer> store =
|
||||||
|
new IdentityHashStore<Key, Integer>(4);
|
||||||
|
store.visitAll(new Visitor<Key, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void accept(Key k, Integer v) {
|
||||||
|
Assert.fail("found key " + k + " in empty IdentityHashStore.");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
Assert.assertTrue(store.isEmpty());
|
||||||
|
Key key1 = new Key("key1");
|
||||||
|
Integer value1 = new Integer(100);
|
||||||
|
Integer value2 = new Integer(200);
|
||||||
|
Integer value3 = new Integer(300);
|
||||||
|
store.put(key1, value1);
|
||||||
|
Key equalToKey1 = new Key("key1");
|
||||||
|
|
||||||
|
// IdentityHashStore compares by object equality, not equals()
|
||||||
|
Assert.assertNull(store.get(equalToKey1));
|
||||||
|
|
||||||
|
Assert.assertTrue(!store.isEmpty());
|
||||||
|
Assert.assertEquals(value1, store.get(key1));
|
||||||
|
store.put(key1, value2);
|
||||||
|
store.put(key1, value3);
|
||||||
|
final List<Integer> allValues = new LinkedList<Integer>();
|
||||||
|
store.visitAll(new Visitor<Key, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void accept(Key k, Integer v) {
|
||||||
|
allValues.add(v);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
Assert.assertEquals(3, allValues.size());
|
||||||
|
for (int i = 0; i < 3; i++) {
|
||||||
|
Integer value = store.remove(key1);
|
||||||
|
Assert.assertTrue(allValues.remove(value));
|
||||||
|
}
|
||||||
|
Assert.assertNull(store.remove(key1));
|
||||||
|
Assert.assertTrue(store.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=60000)
|
||||||
|
public void testAdditionsAndRemovals() {
|
||||||
|
IdentityHashStore<Key, Integer> store =
|
||||||
|
new IdentityHashStore<Key, Integer>(0);
|
||||||
|
final int NUM_KEYS = 1000;
|
||||||
|
LOG.debug("generating " + NUM_KEYS + " keys");
|
||||||
|
final List<Key> keys = new ArrayList<Key>(NUM_KEYS);
|
||||||
|
for (int i = 0; i < NUM_KEYS; i++) {
|
||||||
|
keys.add(new Key("key " + i));
|
||||||
|
}
|
||||||
|
for (int i = 0; i < NUM_KEYS; i++) {
|
||||||
|
store.put(keys.get(i), i);
|
||||||
|
}
|
||||||
|
store.visitAll(new Visitor<Key, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void accept(Key k, Integer v) {
|
||||||
|
Assert.assertTrue(keys.contains(k));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
for (int i = 0; i < NUM_KEYS; i++) {
|
||||||
|
Assert.assertEquals(Integer.valueOf(i),
|
||||||
|
store.remove(keys.get(i)));
|
||||||
|
}
|
||||||
|
store.visitAll(new Visitor<Key, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void accept(Key k, Integer v) {
|
||||||
|
Assert.fail("expected all entries to be removed");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
Assert.assertTrue("expected the store to be " +
|
||||||
|
"empty, but found " + store.numElements() + " elements.",
|
||||||
|
store.isEmpty());
|
||||||
|
Assert.assertEquals(1024, store.capacity());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -45,6 +45,9 @@ HDFS-4949 (Unreleased)
|
||||||
HDFS-5236. Change PathBasedCacheDirective APIs to be a single value
|
HDFS-5236. Change PathBasedCacheDirective APIs to be a single value
|
||||||
rather than batch. (Contributed by Andrew Wang)
|
rather than batch. (Contributed by Andrew Wang)
|
||||||
|
|
||||||
|
HDFS-5191. Revisit zero-copy API in FSDataInputStream to make it more
|
||||||
|
intuitive. (Contributed by Colin Patrick McCabe)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.ByteBufferReadable;
|
import org.apache.hadoop.fs.ByteBufferReadable;
|
||||||
|
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
|
|
||||||
|
@ -28,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
* from a single datanode.
|
* from a single datanode.
|
||||||
*/
|
*/
|
||||||
public interface BlockReader extends ByteBufferReadable {
|
public interface BlockReader extends ByteBufferReadable {
|
||||||
|
|
||||||
|
|
||||||
/* same interface as inputStream java.io.InputStream#read()
|
/* same interface as inputStream java.io.InputStream#read()
|
||||||
* used by DFSInputStream#read()
|
* used by DFSInputStream#read()
|
||||||
|
@ -85,19 +87,12 @@ public interface BlockReader extends ByteBufferReadable {
|
||||||
boolean isShortCircuit();
|
boolean isShortCircuit();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Do a zero-copy read with the current block reader.
|
* Get a ClientMmap object for this BlockReader.
|
||||||
*
|
*
|
||||||
* We assume that the calling code has done bounds checking, and won't ask
|
|
||||||
* us for more bytes than are supposed to be visible (or are in the file).
|
|
||||||
*
|
|
||||||
* @param buffers The zero-copy buffers object.
|
|
||||||
* @param curBlock The current block.
|
* @param curBlock The current block.
|
||||||
* @param blockPos Position in the current block to start reading at.
|
* @return The ClientMmap object, or null if mmap is not
|
||||||
* @param toRead The number of bytes to read from the block.
|
* supported.
|
||||||
*
|
|
||||||
* @return true if the read was done, false otherwise.
|
|
||||||
*/
|
*/
|
||||||
boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||||
LocatedBlock curBlock, long blockPos, int toRead,
|
|
||||||
ClientMmapManager mmapManager);
|
ClientMmapManager mmapManager);
|
||||||
}
|
}
|
||||||
|
|
|
@ -548,46 +548,28 @@ class BlockReaderLocal implements BlockReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean readZeroCopy(HdfsZeroCopyCursor cursor,
|
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||||
LocatedBlock curBlock, long blockPos, int toRead,
|
ClientMmapManager mmapManager) {
|
||||||
ClientMmapManager mmapManager) {
|
|
||||||
if (clientMmap == null) {
|
if (clientMmap == null) {
|
||||||
if (mmapDisabled) {
|
if (mmapDisabled) {
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
clientMmap = mmapManager.fetch(datanodeID, block, dataIn);
|
clientMmap = mmapManager.fetch(datanodeID, block, dataIn);
|
||||||
if (clientMmap == null) {
|
if (clientMmap == null) {
|
||||||
mmapDisabled = true;
|
mmapDisabled = true;
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
LOG.error("Interrupted while setting up mmap for " + filename, e);
|
LOG.error("Interrupted while setting up mmap for " + filename, e);
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
return false;
|
return null;
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("unable to set up mmap for " + filename, e);
|
LOG.error("unable to set up mmap for " + filename, e);
|
||||||
mmapDisabled = true;
|
mmapDisabled = true;
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
long limit = blockPos + toRead;
|
return clientMmap;
|
||||||
if (limit > Integer.MAX_VALUE) {
|
|
||||||
/*
|
|
||||||
* In Java, ByteBuffers use a 32-bit length, capacity, offset, etc.
|
|
||||||
* This limits our mmap'ed regions to 2 GB in length.
|
|
||||||
* TODO: we can implement zero-copy for larger blocks by doing multiple
|
|
||||||
* mmaps.
|
|
||||||
*/
|
|
||||||
mmapDisabled = true;
|
|
||||||
clientMmap.unref();
|
|
||||||
clientMmap = null;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
ByteBuffer mmapBuffer = clientMmap.getMappedByteBuffer().duplicate();
|
|
||||||
mmapBuffer.position((int)blockPos);
|
|
||||||
mmapBuffer.limit((int)limit);
|
|
||||||
cursor.setMmap(clientMmap, mmapBuffer);
|
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,7 @@ import java.util.HashMap;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -705,9 +706,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||||
LocatedBlock curBlock, long blockPos, int toRead,
|
|
||||||
ClientMmapManager mmapManager) {
|
ClientMmapManager mmapManager) {
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -2610,7 +2610,8 @@ public class DFSClient implements java.io.Closeable {
|
||||||
return defaultWriteCachingStrategy;
|
return defaultWriteCachingStrategy;
|
||||||
}
|
}
|
||||||
|
|
||||||
ClientMmapManager getMmapManager() {
|
@VisibleForTesting
|
||||||
|
public ClientMmapManager getMmapManager() {
|
||||||
return mmapManager;
|
return mmapManager;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import java.net.Socket;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.AbstractMap;
|
import java.util.AbstractMap;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.EnumSet;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -36,12 +37,15 @@ import java.util.concurrent.ConcurrentHashMap;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.fs.ByteBufferReadable;
|
import org.apache.hadoop.fs.ByteBufferReadable;
|
||||||
|
import org.apache.hadoop.fs.ByteBufferUtil;
|
||||||
import org.apache.hadoop.fs.CanSetDropBehind;
|
import org.apache.hadoop.fs.CanSetDropBehind;
|
||||||
import org.apache.hadoop.fs.CanSetReadahead;
|
import org.apache.hadoop.fs.CanSetReadahead;
|
||||||
import org.apache.hadoop.fs.ChecksumException;
|
import org.apache.hadoop.fs.ChecksumException;
|
||||||
import org.apache.hadoop.fs.FSInputStream;
|
import org.apache.hadoop.fs.FSInputStream;
|
||||||
|
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
|
||||||
|
import org.apache.hadoop.fs.ReadOption;
|
||||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||||
import org.apache.hadoop.fs.ZeroCopyCursor;
|
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||||
import org.apache.hadoop.hdfs.net.Peer;
|
import org.apache.hadoop.hdfs.net.Peer;
|
||||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||||
|
@ -55,12 +59,14 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
||||||
|
import org.apache.hadoop.io.ByteBufferPool;
|
||||||
import org.apache.hadoop.ipc.RPC;
|
import org.apache.hadoop.ipc.RPC;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.net.unix.DomainSocket;
|
import org.apache.hadoop.net.unix.DomainSocket;
|
||||||
import org.apache.hadoop.security.AccessControlException;
|
import org.apache.hadoop.security.AccessControlException;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
import org.apache.hadoop.util.IdentityHashStore;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
@ -70,7 +76,8 @@ import com.google.common.annotations.VisibleForTesting;
|
||||||
****************************************************************/
|
****************************************************************/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DFSInputStream extends FSInputStream
|
public class DFSInputStream extends FSInputStream
|
||||||
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
||||||
|
HasEnhancedByteBufferAccess {
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static boolean tcpReadsDisabledForTesting = false;
|
static boolean tcpReadsDisabledForTesting = false;
|
||||||
private final PeerCache peerCache;
|
private final PeerCache peerCache;
|
||||||
|
@ -88,6 +95,15 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
||||||
private CachingStrategy cachingStrategy;
|
private CachingStrategy cachingStrategy;
|
||||||
private final ReadStatistics readStatistics = new ReadStatistics();
|
private final ReadStatistics readStatistics = new ReadStatistics();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Track the ByteBuffers that we have handed out to readers.
|
||||||
|
*
|
||||||
|
* The value type can be either ByteBufferPool or ClientMmap, depending on
|
||||||
|
* whether we this is a memory-mapped buffer or not.
|
||||||
|
*/
|
||||||
|
private final IdentityHashStore<ByteBuffer, Object>
|
||||||
|
extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
|
||||||
|
|
||||||
public static class ReadStatistics {
|
public static class ReadStatistics {
|
||||||
public ReadStatistics() {
|
public ReadStatistics() {
|
||||||
this.totalBytesRead = 0;
|
this.totalBytesRead = 0;
|
||||||
|
@ -606,6 +622,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
||||||
}
|
}
|
||||||
dfsClient.checkOpen();
|
dfsClient.checkOpen();
|
||||||
|
|
||||||
|
if (!extendedReadBuffers.isEmpty()) {
|
||||||
|
final StringBuilder builder = new StringBuilder();
|
||||||
|
extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
|
||||||
|
private String prefix = "";
|
||||||
|
@Override
|
||||||
|
public void accept(ByteBuffer k, Object v) {
|
||||||
|
builder.append(prefix).append(k);
|
||||||
|
prefix = ", ";
|
||||||
|
}
|
||||||
|
});
|
||||||
|
DFSClient.LOG.warn("closing file " + src + ", but there are still " +
|
||||||
|
"unreleased ByteBuffers allocated by read(). " +
|
||||||
|
"Please release " + builder.toString() + ".");
|
||||||
|
}
|
||||||
if (blockReader != null) {
|
if (blockReader != null) {
|
||||||
blockReader.close();
|
blockReader.close();
|
||||||
blockReader = null;
|
blockReader = null;
|
||||||
|
@ -1413,9 +1443,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
||||||
closeCurrentBlockReader();
|
closeCurrentBlockReader();
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void readZeroCopy(HdfsZeroCopyCursor zcursor, int toRead)
|
@Override
|
||||||
throws IOException {
|
public synchronized ByteBuffer read(ByteBufferPool bufferPool,
|
||||||
assert(toRead > 0);
|
int maxLength, EnumSet<ReadOption> opts)
|
||||||
|
throws IOException, UnsupportedOperationException {
|
||||||
|
assert(maxLength > 0);
|
||||||
if (((blockReader == null) || (blockEnd == -1)) &&
|
if (((blockReader == null) || (blockEnd == -1)) &&
|
||||||
(pos < getFileLength())) {
|
(pos < getFileLength())) {
|
||||||
/*
|
/*
|
||||||
|
@ -1429,50 +1461,81 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
||||||
"at position " + pos);
|
"at position " + pos);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
boolean canSkipChecksums = opts.contains(ReadOption.SKIP_CHECKSUMS);
|
||||||
|
if (canSkipChecksums) {
|
||||||
|
ByteBuffer buffer = tryReadZeroCopy(maxLength);
|
||||||
|
if (buffer != null) {
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
ByteBuffer buffer = ByteBufferUtil.
|
||||||
|
fallbackRead(this, bufferPool, maxLength);
|
||||||
|
if (buffer != null) {
|
||||||
|
extendedReadBuffers.put(buffer, bufferPool);
|
||||||
|
}
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized ByteBuffer tryReadZeroCopy(int maxLength)
|
||||||
|
throws IOException {
|
||||||
|
// Java ByteBuffers can't be longer than 2 GB, because they use
|
||||||
|
// 4-byte signed integers to represent capacity, etc.
|
||||||
|
// So we can't mmap the parts of the block higher than the 2 GB offset.
|
||||||
|
// FIXME: we could work around this with multiple memory maps.
|
||||||
|
// See HDFS-5101.
|
||||||
|
long blockEnd32 = Math.min(Integer.MAX_VALUE, blockEnd);
|
||||||
long curPos = pos;
|
long curPos = pos;
|
||||||
boolean canSkipChecksums = zcursor.getSkipChecksums();
|
long blockLeft = blockEnd32 - curPos + 1;
|
||||||
long blockLeft = blockEnd - curPos + 1;
|
if (blockLeft <= 0) {
|
||||||
if (zcursor.getAllowShortReads()) {
|
if (DFSClient.LOG.isDebugEnabled()) {
|
||||||
if (blockLeft < toRead) {
|
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
|
||||||
toRead = (int)blockLeft;
|
curPos + " of " + src + "; blockLeft = " + blockLeft +
|
||||||
|
"; blockEnd32 = " + blockEnd32 + ", blockEnd = " + blockEnd +
|
||||||
|
"; maxLength = " + maxLength);
|
||||||
}
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
if (canSkipChecksums && (toRead <= blockLeft)) {
|
int length = Math.min((int)blockLeft, maxLength);
|
||||||
long blockStartInFile = currentLocatedBlock.getStartOffset();
|
long blockStartInFile = currentLocatedBlock.getStartOffset();
|
||||||
long blockPos = curPos - blockStartInFile;
|
long blockPos = curPos - blockStartInFile;
|
||||||
if (blockReader.readZeroCopy(zcursor,
|
long limit = blockPos + length;
|
||||||
currentLocatedBlock, blockPos, toRead,
|
ClientMmap clientMmap =
|
||||||
dfsClient.getMmapManager())) {
|
blockReader.getClientMmap(currentLocatedBlock,
|
||||||
if (DFSClient.LOG.isDebugEnabled()) {
|
dfsClient.getMmapManager());
|
||||||
DFSClient.LOG.debug("readZeroCopy read " + toRead + " bytes from " +
|
if (clientMmap == null) {
|
||||||
"offset " + curPos + " via the zero-copy read path. " +
|
if (DFSClient.LOG.isDebugEnabled()) {
|
||||||
"blockEnd = " + blockEnd);
|
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
|
||||||
}
|
curPos + " of " + src + "; BlockReader#getClientMmap returned " +
|
||||||
readStatistics.addZeroCopyBytes(toRead);
|
"null.");
|
||||||
seek(pos + toRead);
|
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
/*
|
seek(pos + length);
|
||||||
* Slow path reads.
|
ByteBuffer buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
|
||||||
*
|
buffer.position((int)blockPos);
|
||||||
* readStatistics will be updated when we call back into this
|
buffer.limit((int)limit);
|
||||||
* stream's read methods.
|
clientMmap.ref();
|
||||||
*/
|
extendedReadBuffers.put(buffer, clientMmap);
|
||||||
long prevBlockEnd = blockEnd;
|
readStatistics.addZeroCopyBytes(length);
|
||||||
int slowReadAmount = zcursor.readViaSlowPath(toRead);
|
|
||||||
if (DFSClient.LOG.isDebugEnabled()) {
|
if (DFSClient.LOG.isDebugEnabled()) {
|
||||||
DFSClient.LOG.debug("readZeroCopy read " + slowReadAmount + " bytes " +
|
DFSClient.LOG.debug("readZeroCopy read " + maxLength + " bytes from " +
|
||||||
"from offset " + curPos + " via the fallback read path. " +
|
"offset " + curPos + " via the zero-copy read path. " +
|
||||||
"prevBlockEnd = " + prevBlockEnd + ", blockEnd = " + blockEnd +
|
"blockEnd = " + blockEnd);
|
||||||
", canSkipChecksums = " + canSkipChecksums);
|
|
||||||
}
|
}
|
||||||
|
return buffer;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ZeroCopyCursor createZeroCopyCursor()
|
public synchronized void releaseBuffer(ByteBuffer buffer) {
|
||||||
throws IOException, UnsupportedOperationException {
|
Object val = extendedReadBuffers.remove(buffer);
|
||||||
return new HdfsZeroCopyCursor(this,
|
if (val == null) {
|
||||||
dfsClient.getConf().skipShortCircuitChecksums);
|
throw new IllegalArgumentException("tried to release a buffer " +
|
||||||
|
"that was not created by this stream, " + buffer);
|
||||||
|
}
|
||||||
|
if (val instanceof ClientMmap) {
|
||||||
|
((ClientMmap)val).unref();
|
||||||
|
} else if (val instanceof ByteBufferPool) {
|
||||||
|
((ByteBufferPool)val).putBuffer(buffer);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,148 +0,0 @@
|
||||||
/**
|
|
||||||
* 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.hdfs;
|
|
||||||
|
|
||||||
import java.io.EOFException;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.ZeroCopyCursor;
|
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
|
||||||
|
|
||||||
public class HdfsZeroCopyCursor implements ZeroCopyCursor {
|
|
||||||
public static final Log LOG = LogFactory.getLog(HdfsZeroCopyCursor.class);
|
|
||||||
private DFSInputStream stream;
|
|
||||||
private boolean skipChecksums;
|
|
||||||
private boolean allowShortReads;
|
|
||||||
private ClientMmap mmap;
|
|
||||||
private ByteBuffer fallbackBuffer;
|
|
||||||
private ByteBuffer readBuffer;
|
|
||||||
|
|
||||||
HdfsZeroCopyCursor(DFSInputStream stream, boolean skipChecksums) {
|
|
||||||
this.stream = stream;
|
|
||||||
this.skipChecksums = skipChecksums;
|
|
||||||
this.allowShortReads = false;
|
|
||||||
this.mmap = null;
|
|
||||||
this.fallbackBuffer = null;
|
|
||||||
this.readBuffer = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException {
|
|
||||||
stream = null;
|
|
||||||
if (mmap != null) {
|
|
||||||
mmap.unref();
|
|
||||||
mmap = null;
|
|
||||||
}
|
|
||||||
fallbackBuffer = null;
|
|
||||||
readBuffer = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setFallbackBuffer(ByteBuffer fallbackBuffer) {
|
|
||||||
this.fallbackBuffer = fallbackBuffer;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ByteBuffer getFallbackBuffer() {
|
|
||||||
return this.fallbackBuffer;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setSkipChecksums(boolean skipChecksums) {
|
|
||||||
this.skipChecksums = skipChecksums;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean getSkipChecksums() {
|
|
||||||
return this.skipChecksums;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setAllowShortReads(boolean allowShortReads) {
|
|
||||||
this.allowShortReads = allowShortReads;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean getAllowShortReads() {
|
|
||||||
return this.allowShortReads;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void read(int toRead) throws UnsupportedOperationException,
|
|
||||||
EOFException, IOException {
|
|
||||||
if (toRead < 0) {
|
|
||||||
throw new IllegalArgumentException("can't read " + toRead + " bytes.");
|
|
||||||
}
|
|
||||||
stream.readZeroCopy(this, toRead);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ByteBuffer getData() {
|
|
||||||
return readBuffer;
|
|
||||||
}
|
|
||||||
|
|
||||||
int readViaSlowPath(int toRead) throws EOFException, IOException {
|
|
||||||
if (fallbackBuffer == null) {
|
|
||||||
throw new UnsupportedOperationException("unable to read via " +
|
|
||||||
"the fastpath, and there was no fallback buffer provided.");
|
|
||||||
}
|
|
||||||
fallbackBuffer.clear();
|
|
||||||
fallbackBuffer.limit(toRead); // will throw if toRead is too large
|
|
||||||
|
|
||||||
int totalRead = 0;
|
|
||||||
readBuffer = fallbackBuffer;
|
|
||||||
try {
|
|
||||||
while (toRead > 0) {
|
|
||||||
int nread = stream.read(fallbackBuffer);
|
|
||||||
if (nread < 0) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
toRead -= nread;
|
|
||||||
totalRead += nread;
|
|
||||||
if (allowShortReads) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
fallbackBuffer.flip();
|
|
||||||
}
|
|
||||||
if ((toRead > 0) && (!allowShortReads)) {
|
|
||||||
throw new EOFException("only read " + totalRead + " bytes out of " +
|
|
||||||
"a requested " + toRead + " before hitting EOF");
|
|
||||||
}
|
|
||||||
return totalRead;
|
|
||||||
}
|
|
||||||
|
|
||||||
void setMmap(ClientMmap mmap, ByteBuffer readBuffer) {
|
|
||||||
if (this.mmap != mmap) {
|
|
||||||
if (this.mmap != null) {
|
|
||||||
this.mmap.unref();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
this.mmap = mmap;
|
|
||||||
mmap.ref();
|
|
||||||
this.readBuffer = readBuffer;
|
|
||||||
}
|
|
||||||
|
|
||||||
ClientMmap getMmap() {
|
|
||||||
return mmap;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.fs.FSInputChecker;
|
import org.apache.hadoop.fs.FSInputChecker;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||||
import org.apache.hadoop.hdfs.net.Peer;
|
import org.apache.hadoop.hdfs.net.Peer;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
|
@ -489,9 +490,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||||
LocatedBlock curBlock, long blockPos, int toRead,
|
|
||||||
ClientMmapManager mmapManager) {
|
ClientMmapManager mmapManager) {
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import java.nio.channels.ReadableByteChannel;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||||
import org.apache.hadoop.hdfs.net.Peer;
|
import org.apache.hadoop.hdfs.net.Peer;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
|
@ -454,9 +455,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||||
LocatedBlock curBlock, long blockPos, int toRead,
|
|
||||||
ClientMmapManager manager) {
|
ClientMmapManager manager) {
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -361,6 +361,10 @@ public class ClientMmapManager implements Closeable {
|
||||||
}
|
}
|
||||||
waitable.provide(mmap);
|
waitable.provide(mmap);
|
||||||
}
|
}
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.info("created a new ClientMmap for block " + key.block +
|
||||||
|
" on datanode " + key.datanode);
|
||||||
|
}
|
||||||
return mmap;
|
return mmap;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -403,8 +407,10 @@ public class ClientMmapManager implements Closeable {
|
||||||
finally {
|
finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
LOG.debug("reusing existing mmap with datanodeID=" + datanodeID +
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("reusing existing mmap with datanodeID=" + datanodeID +
|
||||||
", " + "block=" + block);
|
", " + "block=" + block);
|
||||||
|
}
|
||||||
return mmap;
|
return mmap;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -32,6 +32,22 @@ int expectFileStats(hdfsFile file,
|
||||||
{
|
{
|
||||||
struct hdfsReadStatistics *stats = NULL;
|
struct hdfsReadStatistics *stats = NULL;
|
||||||
EXPECT_ZERO(hdfsFileGetReadStatistics(file, &stats));
|
EXPECT_ZERO(hdfsFileGetReadStatistics(file, &stats));
|
||||||
|
fprintf(stderr, "expectFileStats(expectedTotalBytesRead=%"PRId64", "
|
||||||
|
"expectedTotalLocalBytesRead=%"PRId64", "
|
||||||
|
"expectedTotalShortCircuitBytesRead=%"PRId64", "
|
||||||
|
"expectedTotalZeroCopyBytesRead=%"PRId64", "
|
||||||
|
"totalBytesRead=%"PRId64", "
|
||||||
|
"totalLocalBytesRead=%"PRId64", "
|
||||||
|
"totalShortCircuitBytesRead=%"PRId64", "
|
||||||
|
"totalZeroCopyBytesRead=%"PRId64")\n",
|
||||||
|
expectedTotalBytesRead,
|
||||||
|
expectedTotalLocalBytesRead,
|
||||||
|
expectedTotalShortCircuitBytesRead,
|
||||||
|
expectedTotalZeroCopyBytesRead,
|
||||||
|
stats->totalBytesRead,
|
||||||
|
stats->totalLocalBytesRead,
|
||||||
|
stats->totalShortCircuitBytesRead,
|
||||||
|
stats->totalZeroCopyBytesRead);
|
||||||
if (expectedTotalBytesRead != UINT64_MAX) {
|
if (expectedTotalBytesRead != UINT64_MAX) {
|
||||||
EXPECT_INT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
|
EXPECT_INT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,7 +39,7 @@
|
||||||
#define JAVA_NET_ISA "java/net/InetSocketAddress"
|
#define JAVA_NET_ISA "java/net/InetSocketAddress"
|
||||||
#define JAVA_NET_URI "java/net/URI"
|
#define JAVA_NET_URI "java/net/URI"
|
||||||
#define JAVA_STRING "java/lang/String"
|
#define JAVA_STRING "java/lang/String"
|
||||||
#define HADOOP_ZERO_COPY_CURSOR "org/apache/hadoop/fs/ZeroCopyCursor"
|
#define READ_OPTION "org/apache/hadoop/fs/ReadOption"
|
||||||
|
|
||||||
#define JAVA_VOID "V"
|
#define JAVA_VOID "V"
|
||||||
|
|
||||||
|
@ -2103,151 +2103,258 @@ int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime)
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
struct hadoopZeroCopyCursor* hadoopZeroCopyCursorAlloc(hdfsFile file)
|
/**
|
||||||
|
* Zero-copy options.
|
||||||
|
*
|
||||||
|
* We cache the EnumSet of ReadOptions which has to be passed into every
|
||||||
|
* readZero call, to avoid reconstructing it each time. This cache is cleared
|
||||||
|
* whenever an element changes.
|
||||||
|
*/
|
||||||
|
struct hadoopRzOptions
|
||||||
{
|
{
|
||||||
int ret;
|
JNIEnv *env;
|
||||||
jobject zcursor = NULL;
|
int skipChecksums;
|
||||||
jvalue jVal;
|
jobject byteBufferPool;
|
||||||
jthrowable jthr;
|
jobject cachedEnumSet;
|
||||||
JNIEnv* env;
|
};
|
||||||
|
|
||||||
|
struct hadoopRzOptions *hadoopRzOptionsAlloc(void)
|
||||||
|
{
|
||||||
|
struct hadoopRzOptions *opts;
|
||||||
|
JNIEnv *env;
|
||||||
|
|
||||||
env = getJNIEnv();
|
env = getJNIEnv();
|
||||||
if (env == NULL) {
|
if (!env) {
|
||||||
|
// Check to make sure the JNI environment is set up properly.
|
||||||
errno = EINTERNAL;
|
errno = EINTERNAL;
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
if (file->type != INPUT) {
|
opts = calloc(1, sizeof(struct hadoopRzOptions));
|
||||||
ret = EINVAL;
|
if (!opts) {
|
||||||
goto done;
|
errno = ENOMEM;
|
||||||
|
return NULL;
|
||||||
}
|
}
|
||||||
jthr = invokeMethod(env, &jVal, INSTANCE, (jobject)file->file, HADOOP_ISTRM,
|
return opts;
|
||||||
"createZeroCopyCursor", "()L"HADOOP_ZERO_COPY_CURSOR";");
|
|
||||||
if (jthr) {
|
|
||||||
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorAlloc: createZeroCopyCursor");
|
|
||||||
goto done;
|
|
||||||
}
|
|
||||||
zcursor = (*env)->NewGlobalRef(env, jVal.l);
|
|
||||||
if (!zcursor) {
|
|
||||||
ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorAlloc: NewGlobalRef");
|
|
||||||
}
|
|
||||||
ret = 0;
|
|
||||||
done:
|
|
||||||
if (ret) {
|
|
||||||
errno = ret;
|
|
||||||
}
|
|
||||||
return (struct hadoopZeroCopyCursor*)zcursor;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
int hadoopZeroCopyCursorSetFallbackBuffer(struct hadoopZeroCopyCursor* zcursor,
|
static void hadoopRzOptionsClearCached(JNIEnv *env,
|
||||||
void *cbuf, uint32_t size)
|
struct hadoopRzOptions *opts)
|
||||||
{
|
{
|
||||||
int ret;
|
if (!opts->cachedEnumSet) {
|
||||||
jobject buffer = NULL;
|
|
||||||
jthrowable jthr;
|
|
||||||
JNIEnv* env;
|
|
||||||
|
|
||||||
env = getJNIEnv();
|
|
||||||
if (env == NULL) {
|
|
||||||
errno = EINTERNAL;
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
buffer = (*env)->NewDirectByteBuffer(env, cbuf, size);
|
|
||||||
if (!buffer) {
|
|
||||||
ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorSetFallbackBuffer: NewDirectByteBuffer("
|
|
||||||
"size=%"PRId32"):", size);
|
|
||||||
goto done;
|
|
||||||
}
|
|
||||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
|
||||||
HADOOP_ZERO_COPY_CURSOR, "setFallbackBuffer",
|
|
||||||
"(Ljava/nio/ByteBuffer;)V", buffer);
|
|
||||||
if (jthr) {
|
|
||||||
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorSetFallbackBuffer: "
|
|
||||||
"FileSystem#setFallbackBuffer");
|
|
||||||
goto done;
|
|
||||||
}
|
|
||||||
ret = 0;
|
|
||||||
done:
|
|
||||||
if (ret) {
|
|
||||||
(*env)->DeleteLocalRef(env, buffer);
|
|
||||||
errno = ret;
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
int hadoopZeroCopyCursorSetSkipChecksums(struct hadoopZeroCopyCursor* zcursor,
|
|
||||||
int skipChecksums)
|
|
||||||
{
|
|
||||||
JNIEnv* env;
|
|
||||||
jthrowable jthr;
|
|
||||||
jboolean shouldSkipChecksums = skipChecksums ? JNI_TRUE : JNI_FALSE;
|
|
||||||
|
|
||||||
env = getJNIEnv();
|
|
||||||
if (env == NULL) {
|
|
||||||
errno = EINTERNAL;
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
|
||||||
HADOOP_ZERO_COPY_CURSOR, "setSkipChecksums", "(Z)V",
|
|
||||||
shouldSkipChecksums);
|
|
||||||
if (jthr) {
|
|
||||||
errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorSetSkipChecksums(): setSkipChecksums failed");
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
int hadoopZeroCopyCursorSetAllowShortReads(
|
|
||||||
struct hadoopZeroCopyCursor* zcursor, int allowShort)
|
|
||||||
{
|
|
||||||
JNIEnv* env;
|
|
||||||
jthrowable jthr;
|
|
||||||
jboolean shouldAllowShort = allowShort ? JNI_TRUE : JNI_FALSE;
|
|
||||||
|
|
||||||
env = getJNIEnv();
|
|
||||||
if (env == NULL) {
|
|
||||||
errno = EINTERNAL;
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
|
||||||
HADOOP_ZERO_COPY_CURSOR, "setAllowShortReads", "(Z)V",
|
|
||||||
shouldAllowShort);
|
|
||||||
if (jthr) {
|
|
||||||
errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorSetAllowShortReads(): setAllowShortReads "
|
|
||||||
"failed");
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
void hadoopZeroCopyCursorFree(struct hadoopZeroCopyCursor *zcursor)
|
|
||||||
{
|
|
||||||
JNIEnv* env;
|
|
||||||
jthrowable jthr;
|
|
||||||
|
|
||||||
env = getJNIEnv();
|
|
||||||
if (env == NULL) {
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
(*env)->DeleteGlobalRef(env, opts->cachedEnumSet);
|
||||||
HADOOP_ZERO_COPY_CURSOR, "close", "()V");
|
opts->cachedEnumSet = NULL;
|
||||||
if (jthr) {
|
}
|
||||||
printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyCursorFree(): close failed");
|
int hadoopRzOptionsSetSkipChecksum(
|
||||||
}
|
struct hadoopRzOptions *opts, int skip)
|
||||||
(*env)->DeleteGlobalRef(env, (jobject)zcursor);
|
{
|
||||||
|
JNIEnv *env;
|
||||||
|
env = getJNIEnv();
|
||||||
|
if (!env) {
|
||||||
|
errno = EINTERNAL;
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
hadoopRzOptionsClearCached(env, opts);
|
||||||
|
opts->skipChecksums = !!skip;
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
int hadoopRzOptionsSetByteBufferPool(
|
||||||
|
struct hadoopRzOptions *opts, const char *className)
|
||||||
|
{
|
||||||
|
JNIEnv *env;
|
||||||
|
jthrowable jthr;
|
||||||
|
jobject byteBufferPool = NULL;
|
||||||
|
|
||||||
|
env = getJNIEnv();
|
||||||
|
if (!env) {
|
||||||
|
errno = EINTERNAL;
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Note: we don't have to call hadoopRzOptionsClearCached in this
|
||||||
|
// function, since the ByteBufferPool is passed separately from the
|
||||||
|
// EnumSet of ReadOptions.
|
||||||
|
|
||||||
|
jthr = constructNewObjectOfClass(env, &byteBufferPool, className, "()V");
|
||||||
|
if (jthr) {
|
||||||
|
printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopRzOptionsSetByteBufferPool(className=%s): ", className);
|
||||||
|
errno = EINVAL;
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
if (opts->byteBufferPool) {
|
||||||
|
// Delete any previous ByteBufferPool we had.
|
||||||
|
(*env)->DeleteGlobalRef(env, opts->byteBufferPool);
|
||||||
|
}
|
||||||
|
opts->byteBufferPool = byteBufferPool;
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
void hadoopRzOptionsFree(struct hadoopRzOptions *opts)
|
||||||
|
{
|
||||||
|
JNIEnv *env;
|
||||||
|
env = getJNIEnv();
|
||||||
|
if (!env) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
hadoopRzOptionsClearCached(env, opts);
|
||||||
|
if (opts->byteBufferPool) {
|
||||||
|
(*env)->DeleteGlobalRef(env, opts->byteBufferPool);
|
||||||
|
opts->byteBufferPool = NULL;
|
||||||
|
}
|
||||||
|
free(opts);
|
||||||
|
}
|
||||||
|
|
||||||
|
struct hadoopRzBuffer
|
||||||
|
{
|
||||||
|
jobject byteBuffer;
|
||||||
|
uint8_t *ptr;
|
||||||
|
int32_t length;
|
||||||
|
int direct;
|
||||||
|
};
|
||||||
|
|
||||||
|
static jthrowable hadoopRzOptionsGetEnumSet(JNIEnv *env,
|
||||||
|
struct hadoopRzOptions *opts, jobject *enumSet)
|
||||||
|
{
|
||||||
|
jthrowable jthr = NULL;
|
||||||
|
jobject enumInst = NULL, enumSetObj = NULL;
|
||||||
|
jvalue jVal;
|
||||||
|
|
||||||
|
if (opts->cachedEnumSet) {
|
||||||
|
// If we cached the value, return it now.
|
||||||
|
*enumSet = opts->cachedEnumSet;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
if (opts->skipChecksums) {
|
||||||
|
jthr = fetchEnumInstance(env, READ_OPTION,
|
||||||
|
"SKIP_CHECKSUMS", &enumInst);
|
||||||
|
if (jthr) {
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
jthr = invokeMethod(env, &jVal, STATIC, NULL,
|
||||||
|
"java/util/EnumSet", "of",
|
||||||
|
"(Ljava/lang/Enum;)Ljava/util/EnumSet;", enumInst);
|
||||||
|
if (jthr) {
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
enumSetObj = jVal.l;
|
||||||
|
} else {
|
||||||
|
jclass clazz = (*env)->FindClass(env, READ_OPTION);
|
||||||
|
if (!clazz) {
|
||||||
|
jthr = newRuntimeError(env, "failed "
|
||||||
|
"to find class for %s", READ_OPTION);
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
jthr = invokeMethod(env, &jVal, STATIC, NULL,
|
||||||
|
"java/util/EnumSet", "noneOf",
|
||||||
|
"(Ljava/lang/Class;)Ljava/util/EnumSet;", clazz);
|
||||||
|
enumSetObj = jVal.l;
|
||||||
|
}
|
||||||
|
// create global ref
|
||||||
|
opts->cachedEnumSet = (*env)->NewGlobalRef(env, enumSetObj);
|
||||||
|
if (!opts->cachedEnumSet) {
|
||||||
|
jthr = getPendingExceptionAndClear(env);
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
*enumSet = opts->cachedEnumSet;
|
||||||
|
jthr = NULL;
|
||||||
|
done:
|
||||||
|
(*env)->DeleteLocalRef(env, enumInst);
|
||||||
|
(*env)->DeleteLocalRef(env, enumSetObj);
|
||||||
|
return jthr;
|
||||||
|
}
|
||||||
|
|
||||||
|
static int hadoopReadZeroExtractBuffer(JNIEnv *env,
|
||||||
|
const struct hadoopRzOptions *opts, struct hadoopRzBuffer *buffer)
|
||||||
|
{
|
||||||
|
int ret;
|
||||||
|
jthrowable jthr;
|
||||||
|
jvalue jVal;
|
||||||
|
uint8_t *directStart;
|
||||||
|
void *mallocBuf = NULL;
|
||||||
|
jint position;
|
||||||
|
jarray array = NULL;
|
||||||
|
|
||||||
|
jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
|
||||||
|
"java/nio/ByteBuffer", "remaining", "()I");
|
||||||
|
if (jthr) {
|
||||||
|
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopReadZeroExtractBuffer: ByteBuffer#remaining failed: ");
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
buffer->length = jVal.i;
|
||||||
|
jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
|
||||||
|
"java/nio/ByteBuffer", "position", "()I");
|
||||||
|
if (jthr) {
|
||||||
|
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopReadZeroExtractBuffer: ByteBuffer#position failed: ");
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
position = jVal.i;
|
||||||
|
directStart = (*env)->GetDirectBufferAddress(env, buffer->byteBuffer);
|
||||||
|
if (directStart) {
|
||||||
|
// Handle direct buffers.
|
||||||
|
buffer->ptr = directStart + position;
|
||||||
|
buffer->direct = 1;
|
||||||
|
ret = 0;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
// Handle indirect buffers.
|
||||||
|
// The JNI docs don't say that GetDirectBufferAddress throws any exceptions
|
||||||
|
// when it fails. However, they also don't clearly say that it doesn't. It
|
||||||
|
// seems safest to clear any pending exceptions here, to prevent problems on
|
||||||
|
// various JVMs.
|
||||||
|
(*env)->ExceptionClear(env);
|
||||||
|
if (!opts->byteBufferPool) {
|
||||||
|
fputs("hadoopReadZeroExtractBuffer: we read through the "
|
||||||
|
"zero-copy path, but failed to get the address of the buffer via "
|
||||||
|
"GetDirectBufferAddress. Please make sure your JVM supports "
|
||||||
|
"GetDirectBufferAddress.\n", stderr);
|
||||||
|
ret = ENOTSUP;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
// Get the backing array object of this buffer.
|
||||||
|
jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
|
||||||
|
"java/nio/ByteBuffer", "array", "()[B");
|
||||||
|
if (jthr) {
|
||||||
|
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopReadZeroExtractBuffer: ByteBuffer#array failed: ");
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
array = jVal.l;
|
||||||
|
if (!array) {
|
||||||
|
fputs("hadoopReadZeroExtractBuffer: ByteBuffer#array returned NULL.",
|
||||||
|
stderr);
|
||||||
|
ret = EIO;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
mallocBuf = malloc(buffer->length);
|
||||||
|
if (!mallocBuf) {
|
||||||
|
fprintf(stderr, "hadoopReadZeroExtractBuffer: failed to allocate %d bytes of memory\n",
|
||||||
|
buffer->length);
|
||||||
|
ret = ENOMEM;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
(*env)->GetByteArrayRegion(env, array, position, buffer->length, mallocBuf);
|
||||||
|
jthr = (*env)->ExceptionOccurred(env);
|
||||||
|
if (jthr) {
|
||||||
|
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopReadZeroExtractBuffer: GetByteArrayRegion failed: ");
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
buffer->ptr = mallocBuf;
|
||||||
|
buffer->direct = 0;
|
||||||
|
ret = 0;
|
||||||
|
|
||||||
|
done:
|
||||||
|
free(mallocBuf);
|
||||||
|
(*env)->DeleteLocalRef(env, array);
|
||||||
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Translate an exception from ZeroCopyCursor#read, translate it into a return
|
|
||||||
* code.
|
|
||||||
*/
|
|
||||||
static int translateZCRException(JNIEnv *env, jthrowable exc)
|
static int translateZCRException(JNIEnv *env, jthrowable exc)
|
||||||
{
|
{
|
||||||
int ret;
|
int ret;
|
||||||
|
@ -2255,16 +2362,12 @@ static int translateZCRException(JNIEnv *env, jthrowable exc)
|
||||||
jthrowable jthr = classNameOfObject(exc, env, &className);
|
jthrowable jthr = classNameOfObject(exc, env, &className);
|
||||||
|
|
||||||
if (jthr) {
|
if (jthr) {
|
||||||
fprintf(stderr, "hadoopZeroCopyRead: unknown "
|
fputs("hadoopReadZero: failed to get class name of "
|
||||||
"exception from read().\n");
|
"exception from read().\n", stderr);
|
||||||
destroyLocalReference(env, jthr);
|
destroyLocalReference(env, exc);
|
||||||
destroyLocalReference(env, jthr);
|
destroyLocalReference(env, jthr);
|
||||||
ret = EIO;
|
ret = EIO;
|
||||||
goto done;
|
goto done;
|
||||||
}
|
|
||||||
if (!strcmp(className, "java.io.EOFException")) {
|
|
||||||
ret = 0; // EOF
|
|
||||||
goto done;
|
|
||||||
}
|
}
|
||||||
if (!strcmp(className, "java.lang.UnsupportedOperationException")) {
|
if (!strcmp(className, "java.lang.UnsupportedOperationException")) {
|
||||||
ret = EPROTONOSUPPORT;
|
ret = EPROTONOSUPPORT;
|
||||||
|
@ -2277,72 +2380,116 @@ done:
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int32_t hadoopZeroCopyRead(struct hadoopZeroCopyCursor *zcursor,
|
struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
|
||||||
int32_t toRead, const void **data)
|
struct hadoopRzOptions *opts, int32_t maxLength)
|
||||||
{
|
{
|
||||||
int32_t ret, nRead = -1;
|
JNIEnv *env;
|
||||||
JNIEnv* env;
|
jthrowable jthr = NULL;
|
||||||
jthrowable jthr;
|
|
||||||
jobject byteBuffer = NULL;
|
|
||||||
uint8_t *addr;
|
|
||||||
jint position;
|
|
||||||
jvalue jVal;
|
jvalue jVal;
|
||||||
|
jobject enumSet = NULL, byteBuffer = NULL;
|
||||||
|
struct hadoopRzBuffer* buffer = NULL;
|
||||||
|
int ret;
|
||||||
|
|
||||||
env = getJNIEnv();
|
env = getJNIEnv();
|
||||||
if (env == NULL) {
|
if (!env) {
|
||||||
errno = EINTERNAL;
|
errno = EINTERNAL;
|
||||||
return -1;
|
return NULL;
|
||||||
}
|
}
|
||||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
if (file->type != INPUT) {
|
||||||
HADOOP_ZERO_COPY_CURSOR, "read", "(I)V", toRead);
|
fputs("Cannot read from a non-InputStream object!\n", stderr);
|
||||||
|
ret = EINVAL;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
buffer = calloc(1, sizeof(struct hadoopRzBuffer));
|
||||||
|
if (!buffer) {
|
||||||
|
ret = ENOMEM;
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
jthr = hadoopRzOptionsGetEnumSet(env, opts, &enumSet);
|
||||||
|
if (jthr) {
|
||||||
|
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopReadZero: hadoopRzOptionsGetEnumSet failed: ");
|
||||||
|
goto done;
|
||||||
|
}
|
||||||
|
jthr = invokeMethod(env, &jVal, INSTANCE, file->file, HADOOP_ISTRM, "read",
|
||||||
|
"(Lorg/apache/hadoop/io/ByteBufferPool;ILjava/util/EnumSet;)"
|
||||||
|
"Ljava/nio/ByteBuffer;", opts->byteBufferPool, maxLength, enumSet);
|
||||||
if (jthr) {
|
if (jthr) {
|
||||||
ret = translateZCRException(env, jthr);
|
ret = translateZCRException(env, jthr);
|
||||||
goto done;
|
goto done;
|
||||||
}
|
}
|
||||||
jthr = invokeMethod(env, &jVal, INSTANCE, (jobject)zcursor,
|
|
||||||
HADOOP_ZERO_COPY_CURSOR, "getData",
|
|
||||||
"()Ljava/nio/ByteBuffer;");
|
|
||||||
if (jthr) {
|
|
||||||
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyRead(toRead=%"PRId32"): getData failed",
|
|
||||||
toRead);
|
|
||||||
goto done;
|
|
||||||
}
|
|
||||||
byteBuffer = jVal.l;
|
byteBuffer = jVal.l;
|
||||||
addr = (*env)->GetDirectBufferAddress(env, byteBuffer);
|
if (!byteBuffer) {
|
||||||
if (!addr) {
|
buffer->byteBuffer = NULL;
|
||||||
fprintf(stderr, "hadoopZeroCopyRead(toRead=%"PRId32"): "
|
buffer->length = 0;
|
||||||
"failed to get direct buffer address.\n", toRead);
|
buffer->ptr = NULL;
|
||||||
ret = EIO;
|
} else {
|
||||||
goto done;
|
buffer->byteBuffer = (*env)->NewGlobalRef(env, byteBuffer);
|
||||||
}
|
if (!buffer->byteBuffer) {
|
||||||
jthr = invokeMethod(env, &jVal, INSTANCE, byteBuffer,
|
ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
|
||||||
"java/nio/ByteBuffer", "position", "()I");
|
"hadoopReadZero: failed to create global ref to ByteBuffer");
|
||||||
if (jthr) {
|
goto done;
|
||||||
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
}
|
||||||
"hadoopZeroCopyRead(toRead=%"PRId32"): ByteBuffer#position "
|
ret = hadoopReadZeroExtractBuffer(env, opts, buffer);
|
||||||
"failed", toRead);
|
if (ret) {
|
||||||
goto done;
|
goto done;
|
||||||
}
|
}
|
||||||
position = jVal.i;
|
|
||||||
jthr = invokeMethod(env, &jVal, INSTANCE, byteBuffer,
|
|
||||||
"java/nio/ByteBuffer", "remaining", "()I");
|
|
||||||
if (jthr) {
|
|
||||||
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
|
||||||
"hadoopZeroCopyRead(toRead=%"PRId32"): ByteBuffer#remaining "
|
|
||||||
"failed", toRead);
|
|
||||||
goto done;
|
|
||||||
}
|
}
|
||||||
ret = 0;
|
ret = 0;
|
||||||
nRead = jVal.i;
|
|
||||||
*data = addr + position;
|
|
||||||
done:
|
done:
|
||||||
(*env)->DeleteLocalRef(env, byteBuffer);
|
(*env)->DeleteLocalRef(env, byteBuffer);
|
||||||
if (nRead == -1) {
|
if (ret) {
|
||||||
|
if (buffer) {
|
||||||
|
if (buffer->byteBuffer) {
|
||||||
|
(*env)->DeleteGlobalRef(env, buffer->byteBuffer);
|
||||||
|
}
|
||||||
|
free(buffer);
|
||||||
|
}
|
||||||
errno = ret;
|
errno = ret;
|
||||||
return -1;
|
return NULL;
|
||||||
|
} else {
|
||||||
|
errno = 0;
|
||||||
}
|
}
|
||||||
return nRead;
|
return buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer)
|
||||||
|
{
|
||||||
|
return buffer->length;
|
||||||
|
}
|
||||||
|
|
||||||
|
const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer)
|
||||||
|
{
|
||||||
|
return buffer->ptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer)
|
||||||
|
{
|
||||||
|
jvalue jVal;
|
||||||
|
jthrowable jthr;
|
||||||
|
JNIEnv* env;
|
||||||
|
|
||||||
|
env = getJNIEnv();
|
||||||
|
if (env == NULL) {
|
||||||
|
errno = EINTERNAL;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (buffer->byteBuffer) {
|
||||||
|
jthr = invokeMethod(env, &jVal, INSTANCE, file->file,
|
||||||
|
HADOOP_ISTRM, "releaseBuffer",
|
||||||
|
"(Ljava/nio/ByteBuffer;)V", buffer->byteBuffer);
|
||||||
|
if (jthr) {
|
||||||
|
printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||||
|
"hadoopRzBufferFree: releaseBuffer failed: ");
|
||||||
|
// even on error, we have to delete the reference.
|
||||||
|
}
|
||||||
|
(*env)->DeleteGlobalRef(env, buffer->byteBuffer);
|
||||||
|
}
|
||||||
|
if (!buffer->direct) {
|
||||||
|
free(buffer->ptr);
|
||||||
|
}
|
||||||
|
memset(buffer, 0, sizeof(*buffer));
|
||||||
|
free(buffer);
|
||||||
}
|
}
|
||||||
|
|
||||||
char***
|
char***
|
||||||
|
|
|
@ -36,6 +36,8 @@
|
||||||
#define EINTERNAL 255
|
#define EINTERNAL 255
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
#define ELASTIC_BYTE_BUFFER_POOL_CLASS \
|
||||||
|
"org/apache/hadoop/io/ElasticByteBufferPool"
|
||||||
|
|
||||||
/** All APIs set errno to meaningful values */
|
/** All APIs set errno to meaningful values */
|
||||||
|
|
||||||
|
@ -65,6 +67,10 @@ extern "C" {
|
||||||
struct hdfsFile_internal;
|
struct hdfsFile_internal;
|
||||||
typedef struct hdfsFile_internal* hdfsFile;
|
typedef struct hdfsFile_internal* hdfsFile;
|
||||||
|
|
||||||
|
struct hadoopRzOptions;
|
||||||
|
|
||||||
|
struct hadoopRzBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Determine if a file is open for read.
|
* Determine if a file is open for read.
|
||||||
*
|
*
|
||||||
|
@ -683,86 +689,104 @@ extern "C" {
|
||||||
int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime);
|
int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a zero-copy cursor object.
|
* Allocate a zero-copy options structure.
|
||||||
*
|
*
|
||||||
* @param file The file to use for zero-copy reads.
|
* You must free all options structures allocated with this function using
|
||||||
|
* hadoopRzOptionsFree.
|
||||||
*
|
*
|
||||||
* @return The zero-copy cursor, or NULL + errno on failure.
|
* @return A zero-copy options structure, or NULL if one could
|
||||||
|
* not be allocated. If NULL is returned, errno will
|
||||||
|
* contain the error number.
|
||||||
*/
|
*/
|
||||||
struct hadoopZeroCopyCursor* hadoopZeroCopyCursorAlloc(hdfsFile file);
|
struct hadoopRzOptions *hadoopRzOptionsAlloc(void);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set the fallback buffer which will be used by the zero copy object.
|
* Determine whether we should skip checksums in read0.
|
||||||
*
|
*
|
||||||
* You are responsible for ensuring that this buffer stays valid until you
|
* @param opts The options structure.
|
||||||
* either set a different buffer by calling this function again, or free the
|
* @param skip Nonzero to skip checksums sometimes; zero to always
|
||||||
* zero-copy cursor.
|
* check them.
|
||||||
*
|
*
|
||||||
* @param zcursor The zero-copy cursor.
|
* @return 0 on success; -1 plus errno on failure.
|
||||||
* @param cbuf The buffer to use.
|
|
||||||
* @param size Size of the buffer.
|
|
||||||
*
|
|
||||||
* @return 0 on success. -1 on error. Errno will be set on
|
|
||||||
* error.
|
|
||||||
*/
|
*/
|
||||||
int hadoopZeroCopyCursorSetFallbackBuffer(
|
int hadoopRzOptionsSetSkipChecksum(
|
||||||
struct hadoopZeroCopyCursor* zcursor, void *cbuf, uint32_t size);
|
struct hadoopRzOptions *opts, int skip);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set whether our cursor should skip checksums or not.
|
* Set the ByteBufferPool to use with read0.
|
||||||
*
|
*
|
||||||
* @param zcursor The cursor
|
* @param opts The options structure.
|
||||||
* @param skipChecksums Nonzero to skip checksums.
|
* @param className If this is NULL, we will not use any
|
||||||
|
* ByteBufferPool. If this is non-NULL, it will be
|
||||||
|
* treated as the name of the pool class to use.
|
||||||
|
* For example, you can use
|
||||||
|
* ELASTIC_BYTE_BUFFER_POOL_CLASS.
|
||||||
*
|
*
|
||||||
* @return -1 on error, 0 otherwise.
|
* @return 0 if the ByteBufferPool class was found and
|
||||||
|
* instantiated;
|
||||||
|
* -1 plus errno otherwise.
|
||||||
*/
|
*/
|
||||||
int hadoopZeroCopyCursorSetSkipChecksums(
|
int hadoopRzOptionsSetByteBufferPool(
|
||||||
struct hadoopZeroCopyCursor* zcursor, int skipChecksums);
|
struct hadoopRzOptions *opts, const char *className);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set whether our cursor should allow short reads to occur.
|
* Free a hadoopRzOptionsFree structure.
|
||||||
* Short reads will always occur if there is not enough data to read
|
|
||||||
* (i.e., at EOF), but normally we don't return them when reading other
|
|
||||||
* parts of the file.
|
|
||||||
*
|
*
|
||||||
* @param zcursor The cursor
|
* @param opts The options structure to free.
|
||||||
* @param skipChecksums Nonzero to skip checksums.
|
* Any associated ByteBufferPool will also be freed.
|
||||||
*
|
|
||||||
* @return -1 on error, 0 otherwise.
|
|
||||||
*/
|
*/
|
||||||
int hadoopZeroCopyCursorSetAllowShortReads(
|
void hadoopRzOptionsFree(struct hadoopRzOptions *opts);
|
||||||
struct hadoopZeroCopyCursor* zcursor, int allowShort);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Free zero-copy cursor.
|
* Perform a byte buffer read.
|
||||||
|
* If possible, this will be a zero-copy (mmap) read.
|
||||||
*
|
*
|
||||||
* This will dispose of the cursor allocated by hadoopZeroCopyCursorAlloc, as
|
* @param file The file to read from.
|
||||||
* well as any memory map that we have created. You must be done with the
|
* @param opts An options structure created by hadoopRzOptionsAlloc.
|
||||||
* data returned from hadoopZeroCopyRead before calling this.
|
* @param maxLength The maximum length to read. We may read fewer bytes
|
||||||
|
* than this length.
|
||||||
*
|
*
|
||||||
* @param zcursor The zero-copy cursor.
|
* @return On success, returns a new hadoopRzBuffer.
|
||||||
|
* This buffer will continue to be valid and readable
|
||||||
|
* until it is released by readZeroBufferFree. Failure to
|
||||||
|
* release a buffer will lead to a memory leak.
|
||||||
|
*
|
||||||
|
* NULL plus an errno code on an error.
|
||||||
|
* errno = EOPNOTSUPP indicates that we could not do a
|
||||||
|
* zero-copy read, and there was no ByteBufferPool
|
||||||
|
* supplied.
|
||||||
*/
|
*/
|
||||||
void hadoopZeroCopyCursorFree(struct hadoopZeroCopyCursor *zcursor);
|
struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
|
||||||
|
struct hadoopRzOptions *opts, int32_t maxLength);
|
||||||
|
|
||||||
/*
|
/**
|
||||||
* Perform a zero-copy read.
|
* Determine the length of the buffer returned from readZero.
|
||||||
*
|
*
|
||||||
* @param zcursor The zero-copy cursor object.
|
* @param buffer a buffer returned from readZero.
|
||||||
* @param toRead The maximum amount to read.
|
* @return the length of the buffer.
|
||||||
* @param data (out param) on succesful return, a pointer to the
|
|
||||||
* data. This pointer will remain valid until the next
|
|
||||||
* call to hadoopZeroCopyRead, or until
|
|
||||||
* hadoopZeroCopyCursorFree is called on zcursor.
|
|
||||||
*
|
|
||||||
* @return -2 if zero-copy is unavailable, and
|
|
||||||
* -1 if there was an error. errno will be the error.
|
|
||||||
* 0 if we hit end-of-file without reading anything.
|
|
||||||
* The positive number of bytes read otherwise. Short
|
|
||||||
* reads will happen only if EOF is reached.
|
|
||||||
* The amount read otherwise.
|
|
||||||
*/
|
*/
|
||||||
int32_t hadoopZeroCopyRead(struct hadoopZeroCopyCursor *zcursor,
|
int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer);
|
||||||
int32_t toRead, const void **data);
|
|
||||||
|
/**
|
||||||
|
* Get a pointer to the raw buffer returned from readZero.
|
||||||
|
*
|
||||||
|
* To find out how many bytes this buffer contains, call
|
||||||
|
* hadoopRzBufferLength.
|
||||||
|
*
|
||||||
|
* @param buffer a buffer returned from readZero.
|
||||||
|
* @return a pointer to the start of the buffer. This will be
|
||||||
|
* NULL when end-of-file has been reached.
|
||||||
|
*/
|
||||||
|
const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Release a buffer obtained through readZero.
|
||||||
|
*
|
||||||
|
* @param file The hdfs stream that created this buffer. This must be
|
||||||
|
* the same stream you called hadoopReadZero on.
|
||||||
|
* @param buffer The buffer to release.
|
||||||
|
*/
|
||||||
|
void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer);
|
||||||
|
|
||||||
#ifdef __cplusplus
|
#ifdef __cplusplus
|
||||||
}
|
}
|
||||||
|
|
|
@ -647,3 +647,34 @@ done:
|
||||||
(*env)->DeleteLocalRef(env, jvalue);
|
(*env)->DeleteLocalRef(env, jvalue);
|
||||||
return jthr;
|
return jthr;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
jthrowable fetchEnumInstance(JNIEnv *env, const char *className,
|
||||||
|
const char *valueName, jobject *out)
|
||||||
|
{
|
||||||
|
jclass clazz;
|
||||||
|
jfieldID fieldId;
|
||||||
|
jobject jEnum;
|
||||||
|
char prettyClass[256];
|
||||||
|
|
||||||
|
clazz = (*env)->FindClass(env, className);
|
||||||
|
if (!clazz) {
|
||||||
|
return newRuntimeError(env, "fetchEnum(%s, %s): failed to find class.",
|
||||||
|
className, valueName);
|
||||||
|
}
|
||||||
|
if (snprintf(prettyClass, sizeof(prettyClass), "L%s;", className)
|
||||||
|
>= sizeof(prettyClass)) {
|
||||||
|
return newRuntimeError(env, "fetchEnum(%s, %s): class name too long.",
|
||||||
|
className, valueName);
|
||||||
|
}
|
||||||
|
fieldId = (*env)->GetStaticFieldID(env, clazz, valueName, prettyClass);
|
||||||
|
if (!fieldId) {
|
||||||
|
return getPendingExceptionAndClear(env);
|
||||||
|
}
|
||||||
|
jEnum = (*env)->GetStaticObjectField(env, clazz, fieldId);
|
||||||
|
if (!jEnum) {
|
||||||
|
return getPendingExceptionAndClear(env);
|
||||||
|
}
|
||||||
|
*out = jEnum;
|
||||||
|
return NULL;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
@ -140,6 +140,21 @@ int javaObjectIsOfClass(JNIEnv *env, jobject obj, const char *name);
|
||||||
jthrowable hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
|
jthrowable hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
|
||||||
const char *key, const char *value);
|
const char *key, const char *value);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Fetch an instance of an Enum.
|
||||||
|
*
|
||||||
|
* @param env The JNI environment.
|
||||||
|
* @param className The enum class name.
|
||||||
|
* @param valueName The name of the enum value
|
||||||
|
* @param out (out param) on success, a local reference to an
|
||||||
|
* instance of the enum object. (Since Java enums are
|
||||||
|
* singletones, this is also the only instance.)
|
||||||
|
*
|
||||||
|
* @return NULL on success; exception otherwise
|
||||||
|
*/
|
||||||
|
jthrowable fetchEnumInstance(JNIEnv *env, const char *className,
|
||||||
|
const char *valueName, jobject *out);
|
||||||
|
|
||||||
#endif /*LIBHDFS_JNI_HELPER_H*/
|
#endif /*LIBHDFS_JNI_HELPER_H*/
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -81,40 +81,49 @@ static void printBuf(const uint8_t *buf, size_t len)
|
||||||
static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
|
static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
|
||||||
{
|
{
|
||||||
hdfsFile file = NULL;
|
hdfsFile file = NULL;
|
||||||
struct hadoopZeroCopyCursor *zcursor = NULL;
|
struct hadoopRzOptions *opts = NULL;
|
||||||
uint8_t *backingBuffer = NULL, *block;
|
struct hadoopRzBuffer *buffer = NULL;
|
||||||
const void *zcPtr;
|
uint8_t *block;
|
||||||
|
|
||||||
file = hdfsOpenFile(fs, fileName, O_RDONLY, 0, 0, 0);
|
file = hdfsOpenFile(fs, fileName, O_RDONLY, 0, 0, 0);
|
||||||
EXPECT_NONNULL(file);
|
EXPECT_NONNULL(file);
|
||||||
zcursor = hadoopZeroCopyCursorAlloc(file);
|
opts = hadoopRzOptionsAlloc();
|
||||||
EXPECT_NONNULL(zcursor);
|
EXPECT_NONNULL(opts);
|
||||||
|
EXPECT_ZERO(hadoopRzOptionsSetSkipChecksum(opts, 1));
|
||||||
/* haven't read anything yet */
|
/* haven't read anything yet */
|
||||||
EXPECT_ZERO(expectFileStats(file, 0LL, 0LL, 0LL, 0LL));
|
EXPECT_ZERO(expectFileStats(file, 0LL, 0LL, 0LL, 0LL));
|
||||||
block = getZeroCopyBlockData(0);
|
block = getZeroCopyBlockData(0);
|
||||||
EXPECT_NONNULL(block);
|
EXPECT_NONNULL(block);
|
||||||
/* first read is half of a block. */
|
/* first read is half of a block. */
|
||||||
|
buffer = hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2);
|
||||||
|
EXPECT_NONNULL(buffer);
|
||||||
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2,
|
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2,
|
||||||
hadoopZeroCopyRead(zcursor,
|
hadoopRzBufferLength(buffer));
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2, &zcPtr));
|
EXPECT_ZERO(memcmp(hadoopRzBufferGet(buffer), block,
|
||||||
EXPECT_ZERO(memcmp(zcPtr, block, TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
||||||
|
hadoopRzBufferFree(file, buffer);
|
||||||
/* read the next half of the block */
|
/* read the next half of the block */
|
||||||
|
buffer = hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2);
|
||||||
|
EXPECT_NONNULL(buffer);
|
||||||
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2,
|
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2,
|
||||||
hadoopZeroCopyRead(zcursor,
|
hadoopRzBufferLength(buffer));
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2, &zcPtr));
|
EXPECT_ZERO(memcmp(hadoopRzBufferGet(buffer),
|
||||||
EXPECT_ZERO(memcmp(zcPtr, block + (TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2),
|
block + (TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2),
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
||||||
|
hadoopRzBufferFree(file, buffer);
|
||||||
free(block);
|
free(block);
|
||||||
EXPECT_ZERO(expectFileStats(file, TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
EXPECT_ZERO(expectFileStats(file, TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE));
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE));
|
||||||
/* Now let's read just a few bytes. */
|
/* Now let's read just a few bytes. */
|
||||||
EXPECT_INT_EQ(SMALL_READ_LEN,
|
buffer = hadoopReadZero(file, opts, SMALL_READ_LEN);
|
||||||
hadoopZeroCopyRead(zcursor, SMALL_READ_LEN, &zcPtr));
|
EXPECT_NONNULL(buffer);
|
||||||
|
EXPECT_INT_EQ(SMALL_READ_LEN, hadoopRzBufferLength(buffer));
|
||||||
block = getZeroCopyBlockData(1);
|
block = getZeroCopyBlockData(1);
|
||||||
EXPECT_NONNULL(block);
|
EXPECT_NONNULL(block);
|
||||||
EXPECT_ZERO(memcmp(block, zcPtr, SMALL_READ_LEN));
|
EXPECT_ZERO(memcmp(block, hadoopRzBufferGet(buffer), SMALL_READ_LEN));
|
||||||
|
hadoopRzBufferFree(file, buffer);
|
||||||
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
|
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
|
||||||
hdfsTell(fs, file));
|
hdfsTell(fs, file));
|
||||||
EXPECT_ZERO(expectFileStats(file,
|
EXPECT_ZERO(expectFileStats(file,
|
||||||
|
@ -123,37 +132,36 @@ static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN));
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN));
|
||||||
|
|
||||||
/* Try to read a full block's worth of data. This will cross the block
|
/* Clear 'skip checksums' and test that we can't do zero-copy reads any
|
||||||
* boundary, which means we have to fall back to non-zero-copy reads.
|
* more. Since there is no ByteBufferPool set, we should fail with
|
||||||
* However, because we don't have a backing buffer, the fallback will fail
|
* EPROTONOSUPPORT.
|
||||||
* with EPROTONOSUPPORT. */
|
*/
|
||||||
EXPECT_INT_EQ(-1,
|
EXPECT_ZERO(hadoopRzOptionsSetSkipChecksum(opts, 0));
|
||||||
hadoopZeroCopyRead(zcursor, TEST_ZEROCOPY_FULL_BLOCK_SIZE, &zcPtr));
|
EXPECT_NULL(hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE));
|
||||||
EXPECT_INT_EQ(EPROTONOSUPPORT, errno);
|
EXPECT_INT_EQ(EPROTONOSUPPORT, errno);
|
||||||
|
|
||||||
/* Now set a backing buffer and try again. It should succeed this time. */
|
/* Now set a ByteBufferPool and try again. It should succeed this time. */
|
||||||
backingBuffer = malloc(ZC_BUF_LEN);
|
EXPECT_ZERO(hadoopRzOptionsSetByteBufferPool(opts,
|
||||||
EXPECT_NONNULL(backingBuffer);
|
ELASTIC_BYTE_BUFFER_POOL_CLASS));
|
||||||
EXPECT_ZERO(hadoopZeroCopyCursorSetFallbackBuffer(zcursor,
|
buffer = hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE);
|
||||||
backingBuffer, ZC_BUF_LEN));
|
EXPECT_NONNULL(buffer);
|
||||||
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE, hadoopRzBufferLength(buffer));
|
||||||
hadoopZeroCopyRead(zcursor, TEST_ZEROCOPY_FULL_BLOCK_SIZE, &zcPtr));
|
|
||||||
EXPECT_ZERO(expectFileStats(file,
|
EXPECT_ZERO(expectFileStats(file,
|
||||||
(2 * TEST_ZEROCOPY_FULL_BLOCK_SIZE) + SMALL_READ_LEN,
|
(2 * TEST_ZEROCOPY_FULL_BLOCK_SIZE) + SMALL_READ_LEN,
|
||||||
(2 * TEST_ZEROCOPY_FULL_BLOCK_SIZE) + SMALL_READ_LEN,
|
(2 * TEST_ZEROCOPY_FULL_BLOCK_SIZE) + SMALL_READ_LEN,
|
||||||
(2 * TEST_ZEROCOPY_FULL_BLOCK_SIZE) + SMALL_READ_LEN,
|
(2 * TEST_ZEROCOPY_FULL_BLOCK_SIZE) + SMALL_READ_LEN,
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN));
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN));
|
||||||
EXPECT_ZERO(memcmp(block + SMALL_READ_LEN, zcPtr,
|
EXPECT_ZERO(memcmp(block + SMALL_READ_LEN, hadoopRzBufferGet(buffer),
|
||||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE - SMALL_READ_LEN));
|
TEST_ZEROCOPY_FULL_BLOCK_SIZE - SMALL_READ_LEN));
|
||||||
free(block);
|
free(block);
|
||||||
block = getZeroCopyBlockData(2);
|
block = getZeroCopyBlockData(2);
|
||||||
EXPECT_NONNULL(block);
|
EXPECT_NONNULL(block);
|
||||||
EXPECT_ZERO(memcmp(block, zcPtr +
|
EXPECT_ZERO(memcmp(block, hadoopRzBufferGet(buffer) +
|
||||||
(TEST_ZEROCOPY_FULL_BLOCK_SIZE - SMALL_READ_LEN), SMALL_READ_LEN));
|
(TEST_ZEROCOPY_FULL_BLOCK_SIZE - SMALL_READ_LEN), SMALL_READ_LEN));
|
||||||
|
hadoopRzBufferFree(file, buffer);
|
||||||
free(block);
|
free(block);
|
||||||
hadoopZeroCopyCursorFree(zcursor);
|
hadoopRzOptionsFree(opts);
|
||||||
EXPECT_ZERO(hdfsCloseFile(fs, file));
|
EXPECT_ZERO(hdfsCloseFile(fs, file));
|
||||||
free(backingBuffer);
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,530 @@
|
||||||
|
/**
|
||||||
|
* 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.File;
|
||||||
|
import java.io.FileInputStream;
|
||||||
|
import java.io.FileOutputStream;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
import java.util.Random;
|
||||||
|
|
||||||
|
import org.apache.commons.lang.SystemUtils;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||||
|
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||||
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.io.ByteBufferPool;
|
||||||
|
import org.apache.hadoop.io.ElasticByteBufferPool;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||||
|
import org.apache.hadoop.net.unix.DomainSocket;
|
||||||
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Assume;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Supplier;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class tests if EnhancedByteBufferAccess works correctly.
|
||||||
|
*/
|
||||||
|
public class TestEnhancedByteBufferAccess {
|
||||||
|
private static final Log LOG =
|
||||||
|
LogFactory.getLog(TestEnhancedByteBufferAccess.class.getName());
|
||||||
|
|
||||||
|
static TemporarySocketDirectory sockDir;
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void init() {
|
||||||
|
sockDir = new TemporarySocketDirectory();
|
||||||
|
DomainSocket.disableBindPathValidation();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static byte[] byteBufferToArray(ByteBuffer buf) {
|
||||||
|
byte resultArray[] = new byte[buf.remaining()];
|
||||||
|
buf.get(resultArray);
|
||||||
|
buf.flip();
|
||||||
|
return resultArray;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HdfsConfiguration initZeroCopyTest() {
|
||||||
|
Assume.assumeTrue(NativeIO.isAvailable());
|
||||||
|
Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
|
||||||
|
HdfsConfiguration conf = new HdfsConfiguration();
|
||||||
|
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3);
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100);
|
||||||
|
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||||
|
new File(sockDir.getDir(),
|
||||||
|
"TestRequestMmapAccess._PORT.sock").getAbsolutePath());
|
||||||
|
conf.setBoolean(DFSConfigKeys.
|
||||||
|
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testZeroCopyReads() throws Exception {
|
||||||
|
HdfsConfiguration conf = initZeroCopyTest();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Path TEST_PATH = new Path("/a");
|
||||||
|
FSDataInputStream fsIn = null;
|
||||||
|
final int TEST_FILE_LENGTH = 12345;
|
||||||
|
|
||||||
|
FileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||||
|
TEST_FILE_LENGTH, (short)1, 7567L);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("unexpected InterruptedException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("unexpected TimeoutException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
}
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||||
|
fsIn.close();
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
ByteBuffer result = fsIn.read(null, 4096,
|
||||||
|
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
Assert.assertEquals(4096, result.remaining());
|
||||||
|
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
||||||
|
Assert.assertEquals(4096,
|
||||||
|
dfsIn.getReadStatistics().getTotalBytesRead());
|
||||||
|
Assert.assertEquals(4096,
|
||||||
|
dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
fsIn.releaseBuffer(result);
|
||||||
|
} finally {
|
||||||
|
if (fsIn != null) fsIn.close();
|
||||||
|
if (fs != null) fs.close();
|
||||||
|
if (cluster != null) cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testShortZeroCopyReads() throws Exception {
|
||||||
|
HdfsConfiguration conf = initZeroCopyTest();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Path TEST_PATH = new Path("/a");
|
||||||
|
FSDataInputStream fsIn = null;
|
||||||
|
final int TEST_FILE_LENGTH = 12345;
|
||||||
|
|
||||||
|
FileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
DFSTestUtil.createFile(fs, TEST_PATH, TEST_FILE_LENGTH, (short)1, 7567L);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("unexpected InterruptedException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("unexpected TimeoutException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
}
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||||
|
fsIn.close();
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
|
||||||
|
// Try to read 8192, but only get 4096 because of the block size.
|
||||||
|
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
||||||
|
ByteBuffer result =
|
||||||
|
dfsIn.read(null, 8192, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
Assert.assertEquals(4096, result.remaining());
|
||||||
|
Assert.assertEquals(4096,
|
||||||
|
dfsIn.getReadStatistics().getTotalBytesRead());
|
||||||
|
Assert.assertEquals(4096,
|
||||||
|
dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
dfsIn.releaseBuffer(result);
|
||||||
|
|
||||||
|
// Try to read 4097, but only get 4096 because of the block size.
|
||||||
|
result =
|
||||||
|
dfsIn.read(null, 4097, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
Assert.assertEquals(4096, result.remaining());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 4096, 8192),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
dfsIn.releaseBuffer(result);
|
||||||
|
} finally {
|
||||||
|
if (fsIn != null) fsIn.close();
|
||||||
|
if (fs != null) fs.close();
|
||||||
|
if (cluster != null) cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testZeroCopyReadsNoFallback() throws Exception {
|
||||||
|
HdfsConfiguration conf = initZeroCopyTest();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Path TEST_PATH = new Path("/a");
|
||||||
|
FSDataInputStream fsIn = null;
|
||||||
|
final int TEST_FILE_LENGTH = 12345;
|
||||||
|
|
||||||
|
FileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||||
|
TEST_FILE_LENGTH, (short)1, 7567L);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("unexpected InterruptedException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("unexpected TimeoutException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
}
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||||
|
fsIn.close();
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
||||||
|
ByteBuffer result;
|
||||||
|
try {
|
||||||
|
result = dfsIn.read(null, 4097, EnumSet.noneOf(ReadOption.class));
|
||||||
|
Assert.fail("expected UnsupportedOperationException");
|
||||||
|
} catch (UnsupportedOperationException e) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
result = dfsIn.read(null, 4096, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
Assert.assertEquals(4096, result.remaining());
|
||||||
|
Assert.assertEquals(4096,
|
||||||
|
dfsIn.getReadStatistics().getTotalBytesRead());
|
||||||
|
Assert.assertEquals(4096,
|
||||||
|
dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
} finally {
|
||||||
|
if (fsIn != null) fsIn.close();
|
||||||
|
if (fs != null) fs.close();
|
||||||
|
if (cluster != null) cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class CountingVisitor
|
||||||
|
implements ClientMmapManager.ClientMmapVisitor {
|
||||||
|
int count = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void accept(ClientMmap mmap) {
|
||||||
|
count++;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void reset() {
|
||||||
|
count = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testZeroCopyMmapCache() throws Exception {
|
||||||
|
HdfsConfiguration conf = initZeroCopyTest();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Path TEST_PATH = new Path("/a");
|
||||||
|
final int TEST_FILE_LENGTH = 16385;
|
||||||
|
final int RANDOM_SEED = 23453;
|
||||||
|
FSDataInputStream fsIn = null;
|
||||||
|
ByteBuffer results[] = { null, null, null, null, null };
|
||||||
|
|
||||||
|
DistributedFileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||||
|
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("unexpected InterruptedException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("unexpected TimeoutException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
}
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||||
|
fsIn.close();
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
final ClientMmapManager mmapManager = fs.getClient().getMmapManager();
|
||||||
|
final CountingVisitor countingVisitor = new CountingVisitor();
|
||||||
|
mmapManager.visitMmaps(countingVisitor);
|
||||||
|
Assert.assertEquals(0, countingVisitor.count);
|
||||||
|
mmapManager.visitEvictable(countingVisitor);
|
||||||
|
Assert.assertEquals(0, countingVisitor.count);
|
||||||
|
results[0] = fsIn.read(null, 4096,
|
||||||
|
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
fsIn.seek(0);
|
||||||
|
results[1] = fsIn.read(null, 4096,
|
||||||
|
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
mmapManager.visitMmaps(countingVisitor);
|
||||||
|
Assert.assertEquals(1, countingVisitor.count);
|
||||||
|
countingVisitor.reset();
|
||||||
|
mmapManager.visitEvictable(countingVisitor);
|
||||||
|
Assert.assertEquals(0, countingVisitor.count);
|
||||||
|
countingVisitor.reset();
|
||||||
|
|
||||||
|
// The mmaps should be of the first block of the file.
|
||||||
|
final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
||||||
|
mmapManager.visitMmaps(new ClientMmapManager.ClientMmapVisitor() {
|
||||||
|
@Override
|
||||||
|
public void accept(ClientMmap mmap) {
|
||||||
|
Assert.assertEquals(firstBlock, mmap.getBlock());
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// Read more blocks.
|
||||||
|
results[2] = fsIn.read(null, 4096,
|
||||||
|
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
results[3] = fsIn.read(null, 4096,
|
||||||
|
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
try {
|
||||||
|
results[4] = fsIn.read(null, 4096,
|
||||||
|
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||||
|
Assert.fail("expected UnsupportedOperationException");
|
||||||
|
} catch (UnsupportedOperationException e) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
|
||||||
|
// we should have 3 mmaps, 0 evictable
|
||||||
|
mmapManager.visitMmaps(countingVisitor);
|
||||||
|
Assert.assertEquals(3, countingVisitor.count);
|
||||||
|
countingVisitor.reset();
|
||||||
|
mmapManager.visitEvictable(countingVisitor);
|
||||||
|
Assert.assertEquals(0, countingVisitor.count);
|
||||||
|
|
||||||
|
// After we close the cursors, the mmaps should be evictable for
|
||||||
|
// a brief period of time. Then, they should be closed (we're
|
||||||
|
// using a very quick timeout)
|
||||||
|
for (ByteBuffer buffer : results) {
|
||||||
|
if (buffer != null) {
|
||||||
|
fsIn.releaseBuffer(buffer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||||
|
public Boolean get() {
|
||||||
|
countingVisitor.reset();
|
||||||
|
try {
|
||||||
|
mmapManager.visitEvictable(countingVisitor);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return (0 == countingVisitor.count);
|
||||||
|
}
|
||||||
|
}, 10, 10000);
|
||||||
|
countingVisitor.reset();
|
||||||
|
mmapManager.visitMmaps(countingVisitor);
|
||||||
|
Assert.assertEquals(0, countingVisitor.count);
|
||||||
|
} finally {
|
||||||
|
if (fsIn != null) fsIn.close();
|
||||||
|
if (fs != null) fs.close();
|
||||||
|
if (cluster != null) cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test HDFS fallback reads. HDFS streams support the ByteBufferReadable
|
||||||
|
* interface.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testHdfsFallbackReads() throws Exception {
|
||||||
|
HdfsConfiguration conf = initZeroCopyTest();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Path TEST_PATH = new Path("/a");
|
||||||
|
final int TEST_FILE_LENGTH = 16385;
|
||||||
|
final int RANDOM_SEED = 23453;
|
||||||
|
FSDataInputStream fsIn = null;
|
||||||
|
|
||||||
|
DistributedFileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||||
|
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("unexpected InterruptedException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("unexpected TimeoutException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
}
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||||
|
fsIn.close();
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
testFallbackImpl(fsIn, original);
|
||||||
|
} finally {
|
||||||
|
if (fsIn != null) fsIn.close();
|
||||||
|
if (fs != null) fs.close();
|
||||||
|
if (cluster != null) cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class RestrictedAllocatingByteBufferPool
|
||||||
|
implements ByteBufferPool {
|
||||||
|
private final boolean direct;
|
||||||
|
|
||||||
|
RestrictedAllocatingByteBufferPool(boolean direct) {
|
||||||
|
this.direct = direct;
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public ByteBuffer getBuffer(boolean direct, int length) {
|
||||||
|
Preconditions.checkArgument(this.direct == direct);
|
||||||
|
return direct ? ByteBuffer.allocateDirect(length) :
|
||||||
|
ByteBuffer.allocate(length);
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public void putBuffer(ByteBuffer buffer) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void testFallbackImpl(InputStream stream,
|
||||||
|
byte original[]) throws Exception {
|
||||||
|
RestrictedAllocatingByteBufferPool bufferPool =
|
||||||
|
new RestrictedAllocatingByteBufferPool(
|
||||||
|
stream instanceof ByteBufferReadable);
|
||||||
|
|
||||||
|
ByteBuffer result = ByteBufferUtil.fallbackRead(stream, bufferPool, 10);
|
||||||
|
Assert.assertEquals(10, result.remaining());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 10),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
|
||||||
|
result = ByteBufferUtil.fallbackRead(stream, bufferPool, 5000);
|
||||||
|
Assert.assertEquals(5000, result.remaining());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 10, 5010),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
|
||||||
|
result = ByteBufferUtil.fallbackRead(stream, bufferPool, 9999999);
|
||||||
|
Assert.assertEquals(11375, result.remaining());
|
||||||
|
Assert.assertArrayEquals(Arrays.copyOfRange(original, 5010, 16385),
|
||||||
|
byteBufferToArray(result));
|
||||||
|
|
||||||
|
result = ByteBufferUtil.fallbackRead(stream, bufferPool, 10);
|
||||||
|
Assert.assertNull(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the {@link ByteBufferUtil#fallbackRead} function directly.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testFallbackRead() throws Exception {
|
||||||
|
HdfsConfiguration conf = initZeroCopyTest();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Path TEST_PATH = new Path("/a");
|
||||||
|
final int TEST_FILE_LENGTH = 16385;
|
||||||
|
final int RANDOM_SEED = 23453;
|
||||||
|
FSDataInputStream fsIn = null;
|
||||||
|
|
||||||
|
DistributedFileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||||
|
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("unexpected InterruptedException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("unexpected TimeoutException during " +
|
||||||
|
"waitReplication: " + e);
|
||||||
|
}
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||||
|
fsIn.close();
|
||||||
|
fsIn = fs.open(TEST_PATH);
|
||||||
|
testFallbackImpl(fsIn, original);
|
||||||
|
} finally {
|
||||||
|
if (fsIn != null) fsIn.close();
|
||||||
|
if (fs != null) fs.close();
|
||||||
|
if (cluster != null) cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test fallback reads on a stream which does not support the
|
||||||
|
* ByteBufferReadable * interface.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testIndirectFallbackReads() throws Exception {
|
||||||
|
final File TEST_DIR = new File(
|
||||||
|
System.getProperty("test.build.data","build/test/data"));
|
||||||
|
final String TEST_PATH = TEST_DIR + File.separator +
|
||||||
|
"indirectFallbackTestFile";
|
||||||
|
final int TEST_FILE_LENGTH = 16385;
|
||||||
|
final int RANDOM_SEED = 23453;
|
||||||
|
FileOutputStream fos = null;
|
||||||
|
FileInputStream fis = null;
|
||||||
|
try {
|
||||||
|
fos = new FileOutputStream(TEST_PATH);
|
||||||
|
Random random = new Random(RANDOM_SEED);
|
||||||
|
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||||
|
random.nextBytes(original);
|
||||||
|
fos.write(original);
|
||||||
|
fos.close();
|
||||||
|
fos = null;
|
||||||
|
fis = new FileInputStream(TEST_PATH);
|
||||||
|
testFallbackImpl(fis, original);
|
||||||
|
} finally {
|
||||||
|
IOUtils.cleanup(LOG, fos, fis);
|
||||||
|
new File(TEST_PATH).delete();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,24 +23,18 @@ import java.io.FileInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.RandomAccessFile;
|
import java.io.RandomAccessFile;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
import org.apache.commons.lang.SystemUtils;
|
|
||||||
import org.apache.hadoop.fs.ChecksumException;
|
import org.apache.hadoop.fs.ChecksumException;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.ZeroCopyCursor;
|
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
|
||||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
|
||||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.net.unix.DomainSocket;
|
import org.apache.hadoop.net.unix.DomainSocket;
|
||||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Assume;
|
import org.junit.Assume;
|
||||||
|
@ -445,322 +439,4 @@ public class TestBlockReaderLocal {
|
||||||
if (sockDir != null) sockDir.close();
|
if (sockDir != null) sockDir.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static byte[] byteBufferToArray(ByteBuffer buf) {
|
|
||||||
byte resultArray[] = new byte[buf.remaining()];
|
|
||||||
buf.get(resultArray);
|
|
||||||
return resultArray;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static HdfsConfiguration initZeroCopyTest() {
|
|
||||||
Assume.assumeTrue(NativeIO.isAvailable());
|
|
||||||
Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
|
|
||||||
HdfsConfiguration conf = new HdfsConfiguration();
|
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
|
||||||
sockDir = new TemporarySocketDirectory();
|
|
||||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
|
|
||||||
conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3);
|
|
||||||
conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100);
|
|
||||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
|
||||||
new File(sockDir.getDir(),
|
|
||||||
"TestRequestMmapAccess._PORT.sock").getAbsolutePath());
|
|
||||||
conf.setBoolean(DFSConfigKeys.
|
|
||||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
|
|
||||||
return conf;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testZeroCopyReads() throws Exception {
|
|
||||||
HdfsConfiguration conf = initZeroCopyTest();
|
|
||||||
MiniDFSCluster cluster = null;
|
|
||||||
final Path TEST_PATH = new Path("/a");
|
|
||||||
FSDataInputStream fsIn = null;
|
|
||||||
ZeroCopyCursor zcursor = null;
|
|
||||||
|
|
||||||
FileSystem fs = null;
|
|
||||||
try {
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
||||||
cluster.waitActive();
|
|
||||||
fs = cluster.getFileSystem();
|
|
||||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
|
||||||
BlockReaderLocalTest.TEST_LENGTH, (short)1, 7567L);
|
|
||||||
try {
|
|
||||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
Assert.fail("unexpected InterruptedException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
Assert.fail("unexpected TimeoutException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
}
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
|
||||||
IOUtils.readFully(fsIn, original, 0,
|
|
||||||
BlockReaderLocalTest.TEST_LENGTH);
|
|
||||||
fsIn.close();
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
zcursor = fsIn.createZeroCopyCursor();
|
|
||||||
zcursor.setFallbackBuffer(ByteBuffer.
|
|
||||||
allocateDirect(1024 * 1024 * 4));
|
|
||||||
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
|
||||||
zcursor.read(4096);
|
|
||||||
ByteBuffer result = zcursor.getData();
|
|
||||||
Assert.assertEquals(4096, result.remaining());
|
|
||||||
Assert.assertEquals(4096,
|
|
||||||
dfsIn.getReadStatistics().getTotalBytesRead());
|
|
||||||
Assert.assertEquals(4096,
|
|
||||||
dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
|
|
||||||
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
|
|
||||||
byteBufferToArray(result));
|
|
||||||
} finally {
|
|
||||||
if (zcursor != null) zcursor.close();
|
|
||||||
if (fsIn != null) fsIn.close();
|
|
||||||
if (fs != null) fs.close();
|
|
||||||
if (cluster != null) cluster.shutdown();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testShortZeroCopyReads() throws Exception {
|
|
||||||
HdfsConfiguration conf = initZeroCopyTest();
|
|
||||||
MiniDFSCluster cluster = null;
|
|
||||||
final Path TEST_PATH = new Path("/a");
|
|
||||||
FSDataInputStream fsIn = null;
|
|
||||||
ZeroCopyCursor zcursor = null;
|
|
||||||
|
|
||||||
FileSystem fs = null;
|
|
||||||
try {
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
||||||
cluster.waitActive();
|
|
||||||
fs = cluster.getFileSystem();
|
|
||||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
|
||||||
BlockReaderLocalTest.TEST_LENGTH, (short)1, 7567L);
|
|
||||||
try {
|
|
||||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
Assert.fail("unexpected InterruptedException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
Assert.fail("unexpected TimeoutException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
}
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
|
||||||
IOUtils.readFully(fsIn, original, 0,
|
|
||||||
BlockReaderLocalTest.TEST_LENGTH);
|
|
||||||
fsIn.close();
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
zcursor = fsIn.createZeroCopyCursor();
|
|
||||||
zcursor.setFallbackBuffer(ByteBuffer.
|
|
||||||
allocateDirect(1024 * 1024 * 4));
|
|
||||||
zcursor.setAllowShortReads(true);
|
|
||||||
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
|
||||||
zcursor.read(8192);
|
|
||||||
ByteBuffer result = zcursor.getData();
|
|
||||||
Assert.assertEquals(4096, result.remaining());
|
|
||||||
Assert.assertEquals(4096,
|
|
||||||
dfsIn.getReadStatistics().getTotalBytesRead());
|
|
||||||
Assert.assertEquals(4096,
|
|
||||||
dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
|
|
||||||
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
|
|
||||||
byteBufferToArray(result));
|
|
||||||
zcursor.read(4097);
|
|
||||||
result = zcursor.getData();
|
|
||||||
Assert.assertEquals(4096, result.remaining());
|
|
||||||
Assert.assertArrayEquals(Arrays.copyOfRange(original, 4096, 8192),
|
|
||||||
byteBufferToArray(result));
|
|
||||||
zcursor.setAllowShortReads(false);
|
|
||||||
zcursor.read(4100);
|
|
||||||
result = zcursor.getData();
|
|
||||||
Assert.assertEquals(4100, result.remaining());
|
|
||||||
|
|
||||||
Assert.assertArrayEquals(Arrays.copyOfRange(original, 8192, 12292),
|
|
||||||
byteBufferToArray(result));
|
|
||||||
} finally {
|
|
||||||
if (zcursor != null) zcursor.close();
|
|
||||||
if (fsIn != null) fsIn.close();
|
|
||||||
if (fs != null) fs.close();
|
|
||||||
if (cluster != null) cluster.shutdown();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testZeroCopyReadsNoBackingBuffer() throws Exception {
|
|
||||||
HdfsConfiguration conf = initZeroCopyTest();
|
|
||||||
MiniDFSCluster cluster = null;
|
|
||||||
final Path TEST_PATH = new Path("/a");
|
|
||||||
FSDataInputStream fsIn = null;
|
|
||||||
ZeroCopyCursor zcursor = null;
|
|
||||||
|
|
||||||
FileSystem fs = null;
|
|
||||||
try {
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
||||||
cluster.waitActive();
|
|
||||||
fs = cluster.getFileSystem();
|
|
||||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
|
||||||
BlockReaderLocalTest.TEST_LENGTH, (short)1, 7567L);
|
|
||||||
try {
|
|
||||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
Assert.fail("unexpected InterruptedException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
Assert.fail("unexpected TimeoutException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
}
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
|
||||||
IOUtils.readFully(fsIn, original, 0,
|
|
||||||
BlockReaderLocalTest.TEST_LENGTH);
|
|
||||||
fsIn.close();
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
zcursor = fsIn.createZeroCopyCursor();
|
|
||||||
zcursor.setAllowShortReads(false);
|
|
||||||
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
|
||||||
// This read is longer than the file, and we do not have short reads enabled.
|
|
||||||
try {
|
|
||||||
zcursor.read(8192);
|
|
||||||
Assert.fail("expected UnsupportedOperationException");
|
|
||||||
} catch (UnsupportedOperationException e) {
|
|
||||||
// expected
|
|
||||||
}
|
|
||||||
// This read is longer than the block, and we do not have short reads enabled.
|
|
||||||
try {
|
|
||||||
zcursor.read(4097);
|
|
||||||
Assert.fail("expected UnsupportedOperationException");
|
|
||||||
} catch (UnsupportedOperationException e) {
|
|
||||||
// expected
|
|
||||||
}
|
|
||||||
// This read should succeed.
|
|
||||||
zcursor.read(4096);
|
|
||||||
ByteBuffer result = zcursor.getData();
|
|
||||||
Assert.assertEquals(4096, result.remaining());
|
|
||||||
Assert.assertEquals(4096,
|
|
||||||
dfsIn.getReadStatistics().getTotalBytesRead());
|
|
||||||
Assert.assertEquals(4096,
|
|
||||||
dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
|
|
||||||
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
|
|
||||||
byteBufferToArray(result));
|
|
||||||
} finally {
|
|
||||||
if (zcursor != null) zcursor.close();
|
|
||||||
if (fsIn != null) fsIn.close();
|
|
||||||
if (fs != null) fs.close();
|
|
||||||
if (cluster != null) cluster.shutdown();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class CountingVisitor
|
|
||||||
implements ClientMmapManager.ClientMmapVisitor {
|
|
||||||
int count = 0;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void accept(ClientMmap mmap) {
|
|
||||||
count++;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void reset() {
|
|
||||||
count = 0;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testZeroCopyMmapCache() throws Exception {
|
|
||||||
HdfsConfiguration conf = initZeroCopyTest();
|
|
||||||
MiniDFSCluster cluster = null;
|
|
||||||
final Path TEST_PATH = new Path("/a");
|
|
||||||
final int TEST_FILE_LENGTH = 16385;
|
|
||||||
final int RANDOM_SEED = 23453;
|
|
||||||
FSDataInputStream fsIn = null;
|
|
||||||
ZeroCopyCursor zcursor[] = { null, null, null, null, null };
|
|
||||||
|
|
||||||
DistributedFileSystem fs = null;
|
|
||||||
try {
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
||||||
cluster.waitActive();
|
|
||||||
fs = cluster.getFileSystem();
|
|
||||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
|
||||||
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
|
||||||
try {
|
|
||||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
Assert.fail("unexpected InterruptedException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
Assert.fail("unexpected TimeoutException during " +
|
|
||||||
"waitReplication: " + e);
|
|
||||||
}
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
byte original[] = new byte[TEST_FILE_LENGTH];
|
|
||||||
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
|
||||||
fsIn.close();
|
|
||||||
fsIn = fs.open(TEST_PATH);
|
|
||||||
for (int i = 0; i < zcursor.length; i++) {
|
|
||||||
zcursor[i] = fsIn.createZeroCopyCursor();
|
|
||||||
zcursor[i].setAllowShortReads(false);
|
|
||||||
}
|
|
||||||
ClientMmapManager mmapManager = fs.getClient().getMmapManager();
|
|
||||||
CountingVisitor countingVisitor = new CountingVisitor();
|
|
||||||
mmapManager.visitMmaps(countingVisitor);
|
|
||||||
Assert.assertEquals(0, countingVisitor.count);
|
|
||||||
mmapManager.visitEvictable(countingVisitor);
|
|
||||||
Assert.assertEquals(0, countingVisitor.count);
|
|
||||||
zcursor[0].read(4096);
|
|
||||||
fsIn.seek(0);
|
|
||||||
zcursor[1].read(4096);
|
|
||||||
mmapManager.visitMmaps(countingVisitor);
|
|
||||||
Assert.assertEquals(1, countingVisitor.count);
|
|
||||||
countingVisitor.reset();
|
|
||||||
mmapManager.visitEvictable(countingVisitor);
|
|
||||||
Assert.assertEquals(0, countingVisitor.count);
|
|
||||||
countingVisitor.reset();
|
|
||||||
|
|
||||||
// The mmaps should be of the first block of the file.
|
|
||||||
final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
|
||||||
mmapManager.visitMmaps(new ClientMmapManager.ClientMmapVisitor() {
|
|
||||||
@Override
|
|
||||||
public void accept(ClientMmap mmap) {
|
|
||||||
Assert.assertEquals(firstBlock, mmap.getBlock());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
// Read more blocks.
|
|
||||||
zcursor[2].read(4096);
|
|
||||||
zcursor[3].read(4096);
|
|
||||||
try {
|
|
||||||
zcursor[4].read(4096);
|
|
||||||
Assert.fail("expected UnsupportedOperationException");
|
|
||||||
} catch (UnsupportedOperationException e) {
|
|
||||||
// expected
|
|
||||||
}
|
|
||||||
|
|
||||||
// we should have 3 mmaps, 0 evictable
|
|
||||||
mmapManager.visitMmaps(countingVisitor);
|
|
||||||
Assert.assertEquals(3, countingVisitor.count);
|
|
||||||
countingVisitor.reset();
|
|
||||||
mmapManager.visitEvictable(countingVisitor);
|
|
||||||
Assert.assertEquals(0, countingVisitor.count);
|
|
||||||
|
|
||||||
// After we close the cursors, the mmaps should be evictable for
|
|
||||||
// a brief period of time. Then, they should be closed (we're
|
|
||||||
// using a very quick timeout)
|
|
||||||
for (int i = 0; i < zcursor.length; i++) {
|
|
||||||
IOUtils.closeStream(zcursor[i]);
|
|
||||||
}
|
|
||||||
while (true) {
|
|
||||||
countingVisitor.reset();
|
|
||||||
mmapManager.visitEvictable(countingVisitor);
|
|
||||||
if (0 == countingVisitor.count) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
countingVisitor.reset();
|
|
||||||
mmapManager.visitMmaps(countingVisitor);
|
|
||||||
Assert.assertEquals(0, countingVisitor.count);
|
|
||||||
} finally {
|
|
||||||
if (fsIn != null) fsIn.close();
|
|
||||||
if (fs != null) fs.close();
|
|
||||||
if (cluster != null) cluster.shutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue