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:
Colin McCabe 2013-09-24 21:40:53 +00:00
parent 552e081a07
commit 9a361c5821
28 changed files with 1933 additions and 977 deletions

View File

@ -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;
}
}

View File

@ -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);
}
}
}

View File

@ -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.");
}
}

View File

@ -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);
}

View File

@ -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,
}

View File

@ -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();
}

View File

@ -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);
}

View File

@ -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.
}
}
}

View File

@ -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]);
}
}
}
}

View File

@ -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());
}
}

View File

@ -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

View File

@ -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);
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -2610,7 +2610,8 @@ public class DFSClient implements java.io.Closeable {
return defaultWriteCachingStrategy;
}
ClientMmapManager getMmapManager() {
@VisibleForTesting
public ClientMmapManager getMmapManager() {
return mmapManager;
}
}

View File

@ -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);
}
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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***

View File

@ -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
}

View File

@ -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;
}

View File

@ -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*/
/**

View File

@ -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;
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}
}