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
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -19,9 +20,13 @@ package org.apache.hadoop.fs;
|
|||
|
||||
import java.io.*;
|
||||
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;
|
||||
import org.apache.hadoop.fs.ByteBufferUtil;
|
||||
import org.apache.hadoop.util.IdentityHashStore;
|
||||
|
||||
/** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
|
||||
* and buffers input through a {@link BufferedInputStream}. */
|
||||
|
@ -30,7 +35,15 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
public class FSDataInputStream extends DataInputStream
|
||||
implements Seekable, PositionedReadable, Closeable,
|
||||
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)
|
||||
throws IOException {
|
||||
super(in);
|
||||
|
@ -169,13 +182,43 @@ public class FSDataInputStream extends DataInputStream
|
|||
}
|
||||
|
||||
@Override
|
||||
public ZeroCopyCursor createZeroCopyCursor()
|
||||
throws IOException, ZeroCopyUnavailableException {
|
||||
public ByteBuffer read(ByteBufferPool bufferPool, int maxLength,
|
||||
EnumSet<ReadOption> opts)
|
||||
throws IOException, UnsupportedOperationException {
|
||||
try {
|
||||
return ((SupportsZeroCopy)in).createZeroCopyCursor();
|
||||
return ((HasEnhancedByteBufferAccess)in).read(bufferPool,
|
||||
maxLength, opts);
|
||||
}
|
||||
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"})
|
||||
@InterfaceStability.Unstable
|
||||
public abstract class FSInputStream extends InputStream
|
||||
implements Seekable, PositionedReadable, SupportsZeroCopy {
|
||||
implements Seekable, PositionedReadable {
|
||||
/**
|
||||
* Seek to the given offset from the start of the file.
|
||||
* The next read() will be from that location. Can't
|
||||
|
@ -88,11 +88,4 @@ public abstract class FSInputStream extends InputStream
|
|||
throws IOException {
|
||||
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
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Supports zero-copy reads.
|
||||
* Options that can be used when reading from a FileSystem.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public interface SupportsZeroCopy {
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public enum ReadOption {
|
||||
/**
|
||||
* Get a zero-copy cursor to use for zero-copy reads.
|
||||
*
|
||||
* @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.
|
||||
* Skip checksums when reading. This option may be useful when reading a file
|
||||
* format that has built-in checksums, or for testing purposes.
|
||||
*/
|
||||
public ZeroCopyCursor createZeroCopyCursor()
|
||||
throws IOException, ZeroCopyUnavailableException;
|
||||
}
|
||||
SKIP_CHECKSUMS,
|
||||
}
|
|
@ -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
|
||||
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
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
|||
import java.io.IOException;
|
||||
|
||||
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.protocol.LocatedBlock;
|
||||
|
||||
|
@ -28,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|||
* from a single datanode.
|
||||
*/
|
||||
public interface BlockReader extends ByteBufferReadable {
|
||||
|
||||
|
||||
/* same interface as inputStream java.io.InputStream#read()
|
||||
* used by DFSInputStream#read()
|
||||
|
@ -85,19 +87,12 @@ public interface BlockReader extends ByteBufferReadable {
|
|||
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 blockPos Position in the current block to start reading at.
|
||||
* @param toRead The number of bytes to read from the block.
|
||||
*
|
||||
* @return true if the read was done, false otherwise.
|
||||
* @return The ClientMmap object, or null if mmap is not
|
||||
* supported.
|
||||
*/
|
||||
boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
||||
LocatedBlock curBlock, long blockPos, int toRead,
|
||||
ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||
ClientMmapManager mmapManager);
|
||||
}
|
||||
|
|
|
@ -548,46 +548,28 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean readZeroCopy(HdfsZeroCopyCursor cursor,
|
||||
LocatedBlock curBlock, long blockPos, int toRead,
|
||||
ClientMmapManager mmapManager) {
|
||||
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||
ClientMmapManager mmapManager) {
|
||||
if (clientMmap == null) {
|
||||
if (mmapDisabled) {
|
||||
return false;
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
clientMmap = mmapManager.fetch(datanodeID, block, dataIn);
|
||||
if (clientMmap == null) {
|
||||
mmapDisabled = true;
|
||||
return false;
|
||||
return null;
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Interrupted while setting up mmap for " + filename, e);
|
||||
Thread.currentThread().interrupt();
|
||||
return false;
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
LOG.error("unable to set up mmap for " + filename, e);
|
||||
mmapDisabled = true;
|
||||
return false;
|
||||
return null;
|
||||
}
|
||||
}
|
||||
long limit = blockPos + toRead;
|
||||
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;
|
||||
return clientMmap;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.HashMap;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -705,9 +706,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
||||
LocatedBlock curBlock, long blockPos, int toRead,
|
||||
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||
ClientMmapManager mmapManager) {
|
||||
return false;
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2610,7 +2610,8 @@ public class DFSClient implements java.io.Closeable {
|
|||
return defaultWriteCachingStrategy;
|
||||
}
|
||||
|
||||
ClientMmapManager getMmapManager() {
|
||||
@VisibleForTesting
|
||||
public ClientMmapManager getMmapManager() {
|
||||
return mmapManager;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.net.Socket;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
|
@ -36,12 +37,15 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.ByteBufferReadable;
|
||||
import org.apache.hadoop.fs.ByteBufferUtil;
|
||||
import org.apache.hadoop.fs.CanSetDropBehind;
|
||||
import org.apache.hadoop.fs.CanSetReadahead;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
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.ZeroCopyCursor;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
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.server.datanode.CachingStrategy;
|
||||
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.RemoteException;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.IdentityHashStore;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
@ -70,7 +76,8 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
****************************************************************/
|
||||
@InterfaceAudience.Private
|
||||
public class DFSInputStream extends FSInputStream
|
||||
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
||||
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
||||
HasEnhancedByteBufferAccess {
|
||||
@VisibleForTesting
|
||||
static boolean tcpReadsDisabledForTesting = false;
|
||||
private final PeerCache peerCache;
|
||||
|
@ -88,6 +95,15 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
|||
private CachingStrategy cachingStrategy;
|
||||
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 ReadStatistics() {
|
||||
this.totalBytesRead = 0;
|
||||
|
@ -606,6 +622,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
|||
}
|
||||
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) {
|
||||
blockReader.close();
|
||||
blockReader = null;
|
||||
|
@ -1413,9 +1443,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
|||
closeCurrentBlockReader();
|
||||
}
|
||||
|
||||
synchronized void readZeroCopy(HdfsZeroCopyCursor zcursor, int toRead)
|
||||
throws IOException {
|
||||
assert(toRead > 0);
|
||||
@Override
|
||||
public synchronized ByteBuffer read(ByteBufferPool bufferPool,
|
||||
int maxLength, EnumSet<ReadOption> opts)
|
||||
throws IOException, UnsupportedOperationException {
|
||||
assert(maxLength > 0);
|
||||
if (((blockReader == null) || (blockEnd == -1)) &&
|
||||
(pos < getFileLength())) {
|
||||
/*
|
||||
|
@ -1429,50 +1461,81 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
|
|||
"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;
|
||||
boolean canSkipChecksums = zcursor.getSkipChecksums();
|
||||
long blockLeft = blockEnd - curPos + 1;
|
||||
if (zcursor.getAllowShortReads()) {
|
||||
if (blockLeft < toRead) {
|
||||
toRead = (int)blockLeft;
|
||||
long blockLeft = blockEnd32 - curPos + 1;
|
||||
if (blockLeft <= 0) {
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
|
||||
curPos + " of " + src + "; blockLeft = " + blockLeft +
|
||||
"; blockEnd32 = " + blockEnd32 + ", blockEnd = " + blockEnd +
|
||||
"; maxLength = " + maxLength);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
if (canSkipChecksums && (toRead <= blockLeft)) {
|
||||
long blockStartInFile = currentLocatedBlock.getStartOffset();
|
||||
long blockPos = curPos - blockStartInFile;
|
||||
if (blockReader.readZeroCopy(zcursor,
|
||||
currentLocatedBlock, blockPos, toRead,
|
||||
dfsClient.getMmapManager())) {
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("readZeroCopy read " + toRead + " bytes from " +
|
||||
"offset " + curPos + " via the zero-copy read path. " +
|
||||
"blockEnd = " + blockEnd);
|
||||
}
|
||||
readStatistics.addZeroCopyBytes(toRead);
|
||||
seek(pos + toRead);
|
||||
return;
|
||||
int length = Math.min((int)blockLeft, maxLength);
|
||||
long blockStartInFile = currentLocatedBlock.getStartOffset();
|
||||
long blockPos = curPos - blockStartInFile;
|
||||
long limit = blockPos + length;
|
||||
ClientMmap clientMmap =
|
||||
blockReader.getClientMmap(currentLocatedBlock,
|
||||
dfsClient.getMmapManager());
|
||||
if (clientMmap == null) {
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
|
||||
curPos + " of " + src + "; BlockReader#getClientMmap returned " +
|
||||
"null.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
/*
|
||||
* Slow path reads.
|
||||
*
|
||||
* readStatistics will be updated when we call back into this
|
||||
* stream's read methods.
|
||||
*/
|
||||
long prevBlockEnd = blockEnd;
|
||||
int slowReadAmount = zcursor.readViaSlowPath(toRead);
|
||||
seek(pos + length);
|
||||
ByteBuffer buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
|
||||
buffer.position((int)blockPos);
|
||||
buffer.limit((int)limit);
|
||||
clientMmap.ref();
|
||||
extendedReadBuffers.put(buffer, clientMmap);
|
||||
readStatistics.addZeroCopyBytes(length);
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("readZeroCopy read " + slowReadAmount + " bytes " +
|
||||
"from offset " + curPos + " via the fallback read path. " +
|
||||
"prevBlockEnd = " + prevBlockEnd + ", blockEnd = " + blockEnd +
|
||||
", canSkipChecksums = " + canSkipChecksums);
|
||||
DFSClient.LOG.debug("readZeroCopy read " + maxLength + " bytes from " +
|
||||
"offset " + curPos + " via the zero-copy read path. " +
|
||||
"blockEnd = " + blockEnd);
|
||||
}
|
||||
return buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZeroCopyCursor createZeroCopyCursor()
|
||||
throws IOException, UnsupportedOperationException {
|
||||
return new HdfsZeroCopyCursor(this,
|
||||
dfsClient.getConf().skipShortCircuitChecksums);
|
||||
public synchronized void releaseBuffer(ByteBuffer buffer) {
|
||||
Object val = extendedReadBuffers.remove(buffer);
|
||||
if (val == null) {
|
||||
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.fs.FSInputChecker;
|
||||
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.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
|
@ -489,9 +490,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
||||
LocatedBlock curBlock, long blockPos, int toRead,
|
||||
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||
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.LogFactory;
|
||||
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.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
|
@ -454,9 +455,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean readZeroCopy(HdfsZeroCopyCursor buffers,
|
||||
LocatedBlock curBlock, long blockPos, int toRead,
|
||||
public ClientMmap getClientMmap(LocatedBlock curBlock,
|
||||
ClientMmapManager manager) {
|
||||
return false;
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -361,6 +361,10 @@ public class ClientMmapManager implements Closeable {
|
|||
}
|
||||
waitable.provide(mmap);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.info("created a new ClientMmap for block " + key.block +
|
||||
" on datanode " + key.datanode);
|
||||
}
|
||||
return mmap;
|
||||
}
|
||||
|
||||
|
@ -403,8 +407,10 @@ public class ClientMmapManager implements Closeable {
|
|||
finally {
|
||||
lock.unlock();
|
||||
}
|
||||
LOG.debug("reusing existing mmap with datanodeID=" + datanodeID +
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("reusing existing mmap with datanodeID=" + datanodeID +
|
||||
", " + "block=" + block);
|
||||
}
|
||||
return mmap;
|
||||
}
|
||||
|
||||
|
|
|
@ -32,6 +32,22 @@ int expectFileStats(hdfsFile file,
|
|||
{
|
||||
struct hdfsReadStatistics *stats = NULL;
|
||||
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) {
|
||||
EXPECT_INT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
|
||||
}
|
||||
|
|
|
@ -39,7 +39,7 @@
|
|||
#define JAVA_NET_ISA "java/net/InetSocketAddress"
|
||||
#define JAVA_NET_URI "java/net/URI"
|
||||
#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"
|
||||
|
||||
|
@ -2103,151 +2103,258 @@ int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime)
|
|||
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;
|
||||
jobject zcursor = NULL;
|
||||
jvalue jVal;
|
||||
jthrowable jthr;
|
||||
JNIEnv* env;
|
||||
JNIEnv *env;
|
||||
int skipChecksums;
|
||||
jobject byteBufferPool;
|
||||
jobject cachedEnumSet;
|
||||
};
|
||||
|
||||
struct hadoopRzOptions *hadoopRzOptionsAlloc(void)
|
||||
{
|
||||
struct hadoopRzOptions *opts;
|
||||
JNIEnv *env;
|
||||
|
||||
env = getJNIEnv();
|
||||
if (env == NULL) {
|
||||
if (!env) {
|
||||
// Check to make sure the JNI environment is set up properly.
|
||||
errno = EINTERNAL;
|
||||
return NULL;
|
||||
}
|
||||
if (file->type != INPUT) {
|
||||
ret = EINVAL;
|
||||
goto done;
|
||||
opts = calloc(1, sizeof(struct hadoopRzOptions));
|
||||
if (!opts) {
|
||||
errno = ENOMEM;
|
||||
return NULL;
|
||||
}
|
||||
jthr = invokeMethod(env, &jVal, INSTANCE, (jobject)file->file, HADOOP_ISTRM,
|
||||
"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;
|
||||
return opts;
|
||||
}
|
||||
|
||||
int hadoopZeroCopyCursorSetFallbackBuffer(struct hadoopZeroCopyCursor* zcursor,
|
||||
void *cbuf, uint32_t size)
|
||||
static void hadoopRzOptionsClearCached(JNIEnv *env,
|
||||
struct hadoopRzOptions *opts)
|
||||
{
|
||||
int ret;
|
||||
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) {
|
||||
if (!opts->cachedEnumSet) {
|
||||
return;
|
||||
}
|
||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
||||
HADOOP_ZERO_COPY_CURSOR, "close", "()V");
|
||||
if (jthr) {
|
||||
printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||
"hadoopZeroCopyCursorFree(): close failed");
|
||||
}
|
||||
(*env)->DeleteGlobalRef(env, (jobject)zcursor);
|
||||
(*env)->DeleteGlobalRef(env, opts->cachedEnumSet);
|
||||
opts->cachedEnumSet = NULL;
|
||||
}
|
||||
|
||||
int hadoopRzOptionsSetSkipChecksum(
|
||||
struct hadoopRzOptions *opts, int skip)
|
||||
{
|
||||
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)
|
||||
{
|
||||
int ret;
|
||||
|
@ -2255,16 +2362,12 @@ static int translateZCRException(JNIEnv *env, jthrowable exc)
|
|||
jthrowable jthr = classNameOfObject(exc, env, &className);
|
||||
|
||||
if (jthr) {
|
||||
fprintf(stderr, "hadoopZeroCopyRead: unknown "
|
||||
"exception from read().\n");
|
||||
destroyLocalReference(env, jthr);
|
||||
fputs("hadoopReadZero: failed to get class name of "
|
||||
"exception from read().\n", stderr);
|
||||
destroyLocalReference(env, exc);
|
||||
destroyLocalReference(env, jthr);
|
||||
ret = EIO;
|
||||
goto done;
|
||||
}
|
||||
if (!strcmp(className, "java.io.EOFException")) {
|
||||
ret = 0; // EOF
|
||||
goto done;
|
||||
}
|
||||
if (!strcmp(className, "java.lang.UnsupportedOperationException")) {
|
||||
ret = EPROTONOSUPPORT;
|
||||
|
@ -2277,72 +2380,116 @@ done:
|
|||
return ret;
|
||||
}
|
||||
|
||||
int32_t hadoopZeroCopyRead(struct hadoopZeroCopyCursor *zcursor,
|
||||
int32_t toRead, const void **data)
|
||||
struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
|
||||
struct hadoopRzOptions *opts, int32_t maxLength)
|
||||
{
|
||||
int32_t ret, nRead = -1;
|
||||
JNIEnv* env;
|
||||
jthrowable jthr;
|
||||
jobject byteBuffer = NULL;
|
||||
uint8_t *addr;
|
||||
jint position;
|
||||
JNIEnv *env;
|
||||
jthrowable jthr = NULL;
|
||||
jvalue jVal;
|
||||
|
||||
jobject enumSet = NULL, byteBuffer = NULL;
|
||||
struct hadoopRzBuffer* buffer = NULL;
|
||||
int ret;
|
||||
|
||||
env = getJNIEnv();
|
||||
if (env == NULL) {
|
||||
if (!env) {
|
||||
errno = EINTERNAL;
|
||||
return -1;
|
||||
return NULL;
|
||||
}
|
||||
jthr = invokeMethod(env, NULL, INSTANCE, (jobject)zcursor,
|
||||
HADOOP_ZERO_COPY_CURSOR, "read", "(I)V", toRead);
|
||||
if (file->type != INPUT) {
|
||||
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) {
|
||||
ret = translateZCRException(env, jthr);
|
||||
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;
|
||||
addr = (*env)->GetDirectBufferAddress(env, byteBuffer);
|
||||
if (!addr) {
|
||||
fprintf(stderr, "hadoopZeroCopyRead(toRead=%"PRId32"): "
|
||||
"failed to get direct buffer address.\n", toRead);
|
||||
ret = EIO;
|
||||
goto done;
|
||||
}
|
||||
jthr = invokeMethod(env, &jVal, INSTANCE, byteBuffer,
|
||||
"java/nio/ByteBuffer", "position", "()I");
|
||||
if (jthr) {
|
||||
ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
|
||||
"hadoopZeroCopyRead(toRead=%"PRId32"): ByteBuffer#position "
|
||||
"failed", toRead);
|
||||
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;
|
||||
if (!byteBuffer) {
|
||||
buffer->byteBuffer = NULL;
|
||||
buffer->length = 0;
|
||||
buffer->ptr = NULL;
|
||||
} else {
|
||||
buffer->byteBuffer = (*env)->NewGlobalRef(env, byteBuffer);
|
||||
if (!buffer->byteBuffer) {
|
||||
ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
|
||||
"hadoopReadZero: failed to create global ref to ByteBuffer");
|
||||
goto done;
|
||||
}
|
||||
ret = hadoopReadZeroExtractBuffer(env, opts, buffer);
|
||||
if (ret) {
|
||||
goto done;
|
||||
}
|
||||
}
|
||||
ret = 0;
|
||||
nRead = jVal.i;
|
||||
*data = addr + position;
|
||||
done:
|
||||
(*env)->DeleteLocalRef(env, byteBuffer);
|
||||
if (nRead == -1) {
|
||||
if (ret) {
|
||||
if (buffer) {
|
||||
if (buffer->byteBuffer) {
|
||||
(*env)->DeleteGlobalRef(env, buffer->byteBuffer);
|
||||
}
|
||||
free(buffer);
|
||||
}
|
||||
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***
|
||||
|
|
|
@ -36,6 +36,8 @@
|
|||
#define EINTERNAL 255
|
||||
#endif
|
||||
|
||||
#define ELASTIC_BYTE_BUFFER_POOL_CLASS \
|
||||
"org/apache/hadoop/io/ElasticByteBufferPool"
|
||||
|
||||
/** All APIs set errno to meaningful values */
|
||||
|
||||
|
@ -65,6 +67,10 @@ extern "C" {
|
|||
struct hdfsFile_internal;
|
||||
typedef struct hdfsFile_internal* hdfsFile;
|
||||
|
||||
struct hadoopRzOptions;
|
||||
|
||||
struct hadoopRzBuffer;
|
||||
|
||||
/**
|
||||
* 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);
|
||||
|
||||
/**
|
||||
* 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
|
||||
* either set a different buffer by calling this function again, or free the
|
||||
* zero-copy cursor.
|
||||
* @param opts The options structure.
|
||||
* @param skip Nonzero to skip checksums sometimes; zero to always
|
||||
* check them.
|
||||
*
|
||||
* @param zcursor The zero-copy cursor.
|
||||
* @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.
|
||||
* @return 0 on success; -1 plus errno on failure.
|
||||
*/
|
||||
int hadoopZeroCopyCursorSetFallbackBuffer(
|
||||
struct hadoopZeroCopyCursor* zcursor, void *cbuf, uint32_t size);
|
||||
int hadoopRzOptionsSetSkipChecksum(
|
||||
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 skipChecksums Nonzero to skip checksums.
|
||||
* @param opts The options structure.
|
||||
* @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(
|
||||
struct hadoopZeroCopyCursor* zcursor, int skipChecksums);
|
||||
int hadoopRzOptionsSetByteBufferPool(
|
||||
struct hadoopRzOptions *opts, const char *className);
|
||||
|
||||
/**
|
||||
* Set whether our cursor should allow short reads to occur.
|
||||
* 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.
|
||||
* Free a hadoopRzOptionsFree structure.
|
||||
*
|
||||
* @param zcursor The cursor
|
||||
* @param skipChecksums Nonzero to skip checksums.
|
||||
*
|
||||
* @return -1 on error, 0 otherwise.
|
||||
* @param opts The options structure to free.
|
||||
* Any associated ByteBufferPool will also be freed.
|
||||
*/
|
||||
int hadoopZeroCopyCursorSetAllowShortReads(
|
||||
struct hadoopZeroCopyCursor* zcursor, int allowShort);
|
||||
void hadoopRzOptionsFree(struct hadoopRzOptions *opts);
|
||||
|
||||
/**
|
||||
* 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
|
||||
* well as any memory map that we have created. You must be done with the
|
||||
* data returned from hadoopZeroCopyRead before calling this.
|
||||
* @param file The file to read from.
|
||||
* @param opts An options structure created by hadoopRzOptionsAlloc.
|
||||
* @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 toRead The maximum amount to read.
|
||||
* @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.
|
||||
* @param buffer a buffer returned from readZero.
|
||||
* @return the length of the buffer.
|
||||
*/
|
||||
int32_t hadoopZeroCopyRead(struct hadoopZeroCopyCursor *zcursor,
|
||||
int32_t toRead, const void **data);
|
||||
int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer);
|
||||
|
||||
/**
|
||||
* 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
|
||||
}
|
||||
|
|
|
@ -647,3 +647,34 @@ done:
|
|||
(*env)->DeleteLocalRef(env, jvalue);
|
||||
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,
|
||||
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*/
|
||||
|
||||
/**
|
||||
|
|
|
@ -81,40 +81,49 @@ static void printBuf(const uint8_t *buf, size_t len)
|
|||
static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
|
||||
{
|
||||
hdfsFile file = NULL;
|
||||
struct hadoopZeroCopyCursor *zcursor = NULL;
|
||||
uint8_t *backingBuffer = NULL, *block;
|
||||
const void *zcPtr;
|
||||
struct hadoopRzOptions *opts = NULL;
|
||||
struct hadoopRzBuffer *buffer = NULL;
|
||||
uint8_t *block;
|
||||
|
||||
file = hdfsOpenFile(fs, fileName, O_RDONLY, 0, 0, 0);
|
||||
EXPECT_NONNULL(file);
|
||||
zcursor = hadoopZeroCopyCursorAlloc(file);
|
||||
EXPECT_NONNULL(zcursor);
|
||||
opts = hadoopRzOptionsAlloc();
|
||||
EXPECT_NONNULL(opts);
|
||||
EXPECT_ZERO(hadoopRzOptionsSetSkipChecksum(opts, 1));
|
||||
/* haven't read anything yet */
|
||||
EXPECT_ZERO(expectFileStats(file, 0LL, 0LL, 0LL, 0LL));
|
||||
block = getZeroCopyBlockData(0);
|
||||
EXPECT_NONNULL(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,
|
||||
hadoopZeroCopyRead(zcursor,
|
||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2, &zcPtr));
|
||||
EXPECT_ZERO(memcmp(zcPtr, block, TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
||||
hadoopRzBufferLength(buffer));
|
||||
EXPECT_ZERO(memcmp(hadoopRzBufferGet(buffer), block,
|
||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
||||
hadoopRzBufferFree(file, buffer);
|
||||
/* 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,
|
||||
hadoopZeroCopyRead(zcursor,
|
||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2, &zcPtr));
|
||||
EXPECT_ZERO(memcmp(zcPtr, block + (TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2),
|
||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
||||
hadoopRzBufferLength(buffer));
|
||||
EXPECT_ZERO(memcmp(hadoopRzBufferGet(buffer),
|
||||
block + (TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2),
|
||||
TEST_ZEROCOPY_FULL_BLOCK_SIZE / 2));
|
||||
hadoopRzBufferFree(file, buffer);
|
||||
free(block);
|
||||
EXPECT_ZERO(expectFileStats(file, 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. */
|
||||
EXPECT_INT_EQ(SMALL_READ_LEN,
|
||||
hadoopZeroCopyRead(zcursor, SMALL_READ_LEN, &zcPtr));
|
||||
buffer = hadoopReadZero(file, opts, SMALL_READ_LEN);
|
||||
EXPECT_NONNULL(buffer);
|
||||
EXPECT_INT_EQ(SMALL_READ_LEN, hadoopRzBufferLength(buffer));
|
||||
block = getZeroCopyBlockData(1);
|
||||
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,
|
||||
hdfsTell(fs, 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));
|
||||
|
||||
/* Try to read a full block's worth of data. This will cross the block
|
||||
* boundary, which means we have to fall back to non-zero-copy reads.
|
||||
* However, because we don't have a backing buffer, the fallback will fail
|
||||
* with EPROTONOSUPPORT. */
|
||||
EXPECT_INT_EQ(-1,
|
||||
hadoopZeroCopyRead(zcursor, TEST_ZEROCOPY_FULL_BLOCK_SIZE, &zcPtr));
|
||||
/* Clear 'skip checksums' and test that we can't do zero-copy reads any
|
||||
* more. Since there is no ByteBufferPool set, we should fail with
|
||||
* EPROTONOSUPPORT.
|
||||
*/
|
||||
EXPECT_ZERO(hadoopRzOptionsSetSkipChecksum(opts, 0));
|
||||
EXPECT_NULL(hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE));
|
||||
EXPECT_INT_EQ(EPROTONOSUPPORT, errno);
|
||||
|
||||
/* Now set a backing buffer and try again. It should succeed this time. */
|
||||
backingBuffer = malloc(ZC_BUF_LEN);
|
||||
EXPECT_NONNULL(backingBuffer);
|
||||
EXPECT_ZERO(hadoopZeroCopyCursorSetFallbackBuffer(zcursor,
|
||||
backingBuffer, ZC_BUF_LEN));
|
||||
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE,
|
||||
hadoopZeroCopyRead(zcursor, TEST_ZEROCOPY_FULL_BLOCK_SIZE, &zcPtr));
|
||||
/* Now set a ByteBufferPool and try again. It should succeed this time. */
|
||||
EXPECT_ZERO(hadoopRzOptionsSetByteBufferPool(opts,
|
||||
ELASTIC_BYTE_BUFFER_POOL_CLASS));
|
||||
buffer = hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE);
|
||||
EXPECT_NONNULL(buffer);
|
||||
EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE, hadoopRzBufferLength(buffer));
|
||||
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,
|
||||
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));
|
||||
free(block);
|
||||
block = getZeroCopyBlockData(2);
|
||||
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));
|
||||
hadoopRzBufferFree(file, buffer);
|
||||
free(block);
|
||||
hadoopZeroCopyCursorFree(zcursor);
|
||||
hadoopRzOptionsFree(opts);
|
||||
EXPECT_ZERO(hdfsCloseFile(fs, file));
|
||||
free(backingBuffer);
|
||||
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.RandomAccessFile;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import org.apache.commons.lang.SystemUtils;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
|
@ -445,322 +439,4 @@ public class TestBlockReaderLocal {
|
|||
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