HDFS-347. DFS read performance suboptimal when client co-located on nodes with data. Contributed by Colin Patrick McCabe.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1467538 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
801b484f97
|
@ -522,6 +522,7 @@
|
|||
<javahClassName>org.apache.hadoop.io.compress.lz4.Lz4Compressor</javahClassName>
|
||||
<javahClassName>org.apache.hadoop.io.compress.lz4.Lz4Decompressor</javahClassName>
|
||||
<javahClassName>org.apache.hadoop.util.NativeCrc32</javahClassName>
|
||||
<javahClassName>org.apache.hadoop.net.unix.DomainSocket</javahClassName>
|
||||
</javahClassNames>
|
||||
<javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
|
||||
</configuration>
|
||||
|
|
|
@ -163,10 +163,10 @@ add_executable(test_bulk_crc32
|
|||
${D}/util/bulk_crc32.c
|
||||
${T}/util/test_bulk_crc32.c
|
||||
)
|
||||
set_property(SOURCE main.cpp PROPERTY INCLUDE_DIRECTORIES "\"-Werror\" \"-Wall\"")
|
||||
|
||||
SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
|
||||
add_dual_library(hadoop
|
||||
main/native/src/exception.c
|
||||
${D}/io/compress/lz4/Lz4Compressor.c
|
||||
${D}/io/compress/lz4/Lz4Decompressor.c
|
||||
${D}/io/compress/lz4/lz4.c
|
||||
|
@ -177,6 +177,7 @@ add_dual_library(hadoop
|
|||
${D}/io/nativeio/NativeIO.c
|
||||
${D}/io/nativeio/errno_enum.c
|
||||
${D}/io/nativeio/file_descriptor.c
|
||||
${D}/net/unix/DomainSocket.c
|
||||
${D}/security/JniBasedUnixGroupsMapping.c
|
||||
${D}/security/JniBasedUnixGroupsNetgroupMapping.c
|
||||
${D}/security/getGroup.c
|
||||
|
|
|
@ -743,6 +743,21 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
return value.trim();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value of the <code>name</code> property as a trimmed <code>String</code>,
|
||||
* <code>defaultValue</code> if no such property exists.
|
||||
* See @{Configuration#getTrimmed} for more details.
|
||||
*
|
||||
* @param name the property name.
|
||||
* @param defaultValue the property default value.
|
||||
* @return the value of the <code>name</code> or defaultValue
|
||||
* if it is not set.
|
||||
*/
|
||||
public String getTrimmed(String name, String defaultValue) {
|
||||
String ret = getTrimmed(name);
|
||||
return ret == null ? defaultValue : ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value of the <code>name</code> property, without doing
|
||||
|
@ -877,7 +892,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the value of the <code>name</code> property as an <code>int</code>.
|
||||
*
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import java.io.InputStream;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketTimeoutException;
|
||||
|
@ -37,7 +38,8 @@ import java.nio.channels.SelectionKey;
|
|||
* IllegalBlockingModeException.
|
||||
* Please use {@link SocketOutputStream} for writing.
|
||||
*/
|
||||
class SocketInputStream extends InputStream
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
public class SocketInputStream extends InputStream
|
||||
implements ReadableByteChannel {
|
||||
|
||||
private Reader reader;
|
||||
|
|
|
@ -260,4 +260,8 @@ public class SocketOutputStream extends OutputStream
|
|||
throws IOException {
|
||||
transferToFully(fileCh, position, count, null, null);
|
||||
}
|
||||
|
||||
public void setTimeout(int timeoutMs) {
|
||||
writer.setTimeout(timeoutMs);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,704 @@
|
|||
/**
|
||||
* 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.net.unix;
|
||||
|
||||
import java.io.Closeable;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import java.io.FileDescriptor;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.SocketException;
|
||||
import java.nio.channels.AsynchronousCloseException;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.lang.SystemUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.util.NativeCodeLoader;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* The implementation of UNIX domain sockets in Java.
|
||||
*
|
||||
* See {@link DomainSocket} for more information about UNIX domain sockets.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
public class DomainSocket implements Closeable {
|
||||
static {
|
||||
if (SystemUtils.IS_OS_WINDOWS) {
|
||||
loadingFailureReason = "UNIX Domain sockets are not available on Windows.";
|
||||
} else if (!NativeCodeLoader.isNativeCodeLoaded()) {
|
||||
loadingFailureReason = "libhadoop cannot be loaded.";
|
||||
} else {
|
||||
String problem;
|
||||
try {
|
||||
anchorNative();
|
||||
problem = null;
|
||||
} catch (Throwable t) {
|
||||
problem = "DomainSocket#anchorNative got error: " + t.getMessage();
|
||||
}
|
||||
loadingFailureReason = problem;
|
||||
}
|
||||
}
|
||||
|
||||
static Log LOG = LogFactory.getLog(DomainSocket.class);
|
||||
|
||||
/**
|
||||
* True only if we should validate the paths used in {@link DomainSocket#bind()}
|
||||
*/
|
||||
private static boolean validateBindPaths = true;
|
||||
|
||||
/**
|
||||
* The reason why DomainSocket is not available, or null if it is available.
|
||||
*/
|
||||
private final static String loadingFailureReason;
|
||||
|
||||
/**
|
||||
* Initialize the native library code.
|
||||
*/
|
||||
private static native void anchorNative();
|
||||
|
||||
/**
|
||||
* This function is designed to validate that the path chosen for a UNIX
|
||||
* domain socket is secure. A socket path is secure if it doesn't allow
|
||||
* unprivileged users to perform a man-in-the-middle attack against it.
|
||||
* For example, one way to perform a man-in-the-middle attack would be for
|
||||
* a malicious user to move the server socket out of the way and create his
|
||||
* own socket in the same place. Not good.
|
||||
*
|
||||
* Note that we only check the path once. It's possible that the
|
||||
* permissions on the path could change, perhaps to something more relaxed,
|
||||
* immediately after the path passes our validation test-- hence creating a
|
||||
* security hole. However, the purpose of this check is to spot common
|
||||
* misconfigurations. System administrators do not commonly change
|
||||
* permissions on these paths while the server is running.
|
||||
*
|
||||
* @param path the path to validate
|
||||
* @param skipComponents the number of starting path components to skip
|
||||
* validation for (used only for testing)
|
||||
*/
|
||||
@VisibleForTesting
|
||||
native static void validateSocketPathSecurity0(String path,
|
||||
int skipComponents) throws IOException;
|
||||
|
||||
/**
|
||||
* Return true only if UNIX domain sockets are available.
|
||||
*/
|
||||
public static String getLoadingFailureReason() {
|
||||
return loadingFailureReason;
|
||||
}
|
||||
|
||||
/**
|
||||
* Disable validation of the server bind paths.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static void disableBindPathValidation() {
|
||||
validateBindPaths = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a path and a port, compute the effective path by replacing
|
||||
* occurrences of _PORT with the port. This is mainly to make it
|
||||
* possible to run multiple DataNodes locally for testing purposes.
|
||||
*
|
||||
* @param path The source path
|
||||
* @param port Port number to use
|
||||
*
|
||||
* @return The effective path
|
||||
*/
|
||||
public static String getEffectivePath(String path, int port) {
|
||||
return path.replace("_PORT", String.valueOf(port));
|
||||
}
|
||||
|
||||
/**
|
||||
* Tracks the reference count of the file descriptor, and also whether it is
|
||||
* open or closed.
|
||||
*/
|
||||
private static class Status {
|
||||
/**
|
||||
* Bit mask representing a closed domain socket.
|
||||
*/
|
||||
private static final int STATUS_CLOSED_MASK = 1 << 30;
|
||||
|
||||
/**
|
||||
* Status bits
|
||||
*
|
||||
* Bit 30: 0 = DomainSocket open, 1 = DomainSocket closed
|
||||
* Bits 29 to 0: the reference count.
|
||||
*/
|
||||
private final AtomicInteger bits = new AtomicInteger(0);
|
||||
|
||||
Status() { }
|
||||
|
||||
/**
|
||||
* Increment the reference count of the underlying file descriptor.
|
||||
*
|
||||
* @throws ClosedChannelException If the file descriptor is closed.
|
||||
*/
|
||||
void reference() throws ClosedChannelException {
|
||||
int curBits = bits.incrementAndGet();
|
||||
if ((curBits & STATUS_CLOSED_MASK) != 0) {
|
||||
bits.decrementAndGet();
|
||||
throw new ClosedChannelException();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Decrement the reference count of the underlying file descriptor.
|
||||
*
|
||||
* @param checkClosed Whether to throw an exception if the file
|
||||
* descriptor is closed.
|
||||
*
|
||||
* @throws AsynchronousCloseException If the file descriptor is closed and
|
||||
* checkClosed is set.
|
||||
*/
|
||||
void unreference(boolean checkClosed) throws AsynchronousCloseException {
|
||||
int newCount = bits.decrementAndGet();
|
||||
assert (newCount & ~STATUS_CLOSED_MASK) >= 0;
|
||||
if (checkClosed && ((newCount & STATUS_CLOSED_MASK) != 0)) {
|
||||
throw new AsynchronousCloseException();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if the file descriptor is currently open.
|
||||
*
|
||||
* @return True if the file descriptor is currently open.
|
||||
*/
|
||||
boolean isOpen() {
|
||||
return ((bits.get() & STATUS_CLOSED_MASK) == 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark the file descriptor as closed.
|
||||
*
|
||||
* Once the file descriptor is closed, it cannot be reopened.
|
||||
*
|
||||
* @return The current reference count.
|
||||
* @throws ClosedChannelException If someone else closes the file
|
||||
* descriptor before we do.
|
||||
*/
|
||||
int setClosed() throws ClosedChannelException {
|
||||
while (true) {
|
||||
int curBits = bits.get();
|
||||
if ((curBits & STATUS_CLOSED_MASK) != 0) {
|
||||
throw new ClosedChannelException();
|
||||
}
|
||||
if (bits.compareAndSet(curBits, curBits | STATUS_CLOSED_MASK)) {
|
||||
return curBits & (~STATUS_CLOSED_MASK);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the current reference count.
|
||||
*
|
||||
* @return The current reference count.
|
||||
*/
|
||||
int getReferenceCount() {
|
||||
return bits.get() & (~STATUS_CLOSED_MASK);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The socket status.
|
||||
*/
|
||||
private final Status status;
|
||||
|
||||
/**
|
||||
* The file descriptor associated with this UNIX domain socket.
|
||||
*/
|
||||
private final int fd;
|
||||
|
||||
/**
|
||||
* The path associated with this UNIX domain socket.
|
||||
*/
|
||||
private final String path;
|
||||
|
||||
/**
|
||||
* The InputStream associated with this socket.
|
||||
*/
|
||||
private final DomainInputStream inputStream = new DomainInputStream();
|
||||
|
||||
/**
|
||||
* The OutputStream associated with this socket.
|
||||
*/
|
||||
private final DomainOutputStream outputStream = new DomainOutputStream();
|
||||
|
||||
/**
|
||||
* The Channel associated with this socket.
|
||||
*/
|
||||
private final DomainChannel channel = new DomainChannel();
|
||||
|
||||
private DomainSocket(String path, int fd) {
|
||||
this.status = new Status();
|
||||
this.fd = fd;
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
private static native int bind0(String path) throws IOException;
|
||||
|
||||
/**
|
||||
* Create a new DomainSocket listening on the given path.
|
||||
*
|
||||
* @param path The path to bind and listen on.
|
||||
* @return The new DomainSocket.
|
||||
*/
|
||||
public static DomainSocket bindAndListen(String path) throws IOException {
|
||||
if (loadingFailureReason != null) {
|
||||
throw new UnsupportedOperationException(loadingFailureReason);
|
||||
}
|
||||
if (validateBindPaths) {
|
||||
validateSocketPathSecurity0(path, 0);
|
||||
}
|
||||
int fd = bind0(path);
|
||||
return new DomainSocket(path, fd);
|
||||
}
|
||||
|
||||
private static native int accept0(int fd) throws IOException;
|
||||
|
||||
/**
|
||||
* Accept a new UNIX domain connection.
|
||||
*
|
||||
* This method can only be used on sockets that were bound with bind().
|
||||
*
|
||||
* @return The new connection.
|
||||
* @throws IOException If there was an I/O error
|
||||
* performing the accept-- such as the
|
||||
* socket being closed from under us.
|
||||
* @throws SocketTimeoutException If the accept timed out.
|
||||
*/
|
||||
public DomainSocket accept() throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
DomainSocket ret = new DomainSocket(path, accept0(fd));
|
||||
exc = false;
|
||||
return ret;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private static native int connect0(String path);
|
||||
|
||||
/**
|
||||
* Create a new DomainSocket connected to the given path.
|
||||
*
|
||||
* @param path The path to connect to.
|
||||
* @return The new DomainSocket.
|
||||
*/
|
||||
public static DomainSocket connect(String path) throws IOException {
|
||||
if (loadingFailureReason != null) {
|
||||
throw new UnsupportedOperationException(loadingFailureReason);
|
||||
}
|
||||
int fd = connect0(path);
|
||||
return new DomainSocket(path, fd);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if the file descriptor is currently open.
|
||||
*
|
||||
* @return True if the file descriptor is currently open.
|
||||
*/
|
||||
public boolean isOpen() {
|
||||
return status.isOpen();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The socket path.
|
||||
*/
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The socket InputStream
|
||||
*/
|
||||
public DomainInputStream getInputStream() {
|
||||
return inputStream;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The socket OutputStream
|
||||
*/
|
||||
public DomainOutputStream getOutputStream() {
|
||||
return outputStream;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The socket Channel
|
||||
*/
|
||||
public DomainChannel getChannel() {
|
||||
return channel;
|
||||
}
|
||||
|
||||
public static final int SEND_BUFFER_SIZE = 1;
|
||||
public static final int RECEIVE_BUFFER_SIZE = 2;
|
||||
public static final int SEND_TIMEOUT = 3;
|
||||
public static final int RECEIVE_TIMEOUT = 4;
|
||||
|
||||
private static native void setAttribute0(int fd, int type, int val)
|
||||
throws IOException;
|
||||
|
||||
public void setAttribute(int type, int size) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
setAttribute0(fd, type, size);
|
||||
exc = false;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private native int getAttribute0(int fd, int type) throws IOException;
|
||||
|
||||
public int getAttribute(int type) throws IOException {
|
||||
status.reference();
|
||||
int attribute;
|
||||
boolean exc = true;
|
||||
try {
|
||||
attribute = getAttribute0(fd, type);
|
||||
exc = false;
|
||||
return attribute;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private static native void close0(int fd) throws IOException;
|
||||
|
||||
private static native void closeFileDescriptor0(FileDescriptor fd)
|
||||
throws IOException;
|
||||
|
||||
private static native void shutdown0(int fd) throws IOException;
|
||||
|
||||
/**
|
||||
* Close the Socket.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// Set the closed bit on this DomainSocket
|
||||
int refCount;
|
||||
try {
|
||||
refCount = status.setClosed();
|
||||
} catch (ClosedChannelException e) {
|
||||
// Someone else already closed the DomainSocket.
|
||||
return;
|
||||
}
|
||||
// Wait for all references to go away
|
||||
boolean didShutdown = false;
|
||||
boolean interrupted = false;
|
||||
while (refCount > 0) {
|
||||
if (!didShutdown) {
|
||||
try {
|
||||
// Calling shutdown on the socket will interrupt blocking system
|
||||
// calls like accept, write, and read that are going on in a
|
||||
// different thread.
|
||||
shutdown0(fd);
|
||||
} catch (IOException e) {
|
||||
LOG.error("shutdown error: ", e);
|
||||
}
|
||||
didShutdown = true;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException e) {
|
||||
interrupted = true;
|
||||
}
|
||||
refCount = status.getReferenceCount();
|
||||
}
|
||||
|
||||
// At this point, nobody has a reference to the file descriptor,
|
||||
// and nobody will be able to get one in the future either.
|
||||
// We now call close(2) on the file descriptor.
|
||||
// After this point, the file descriptor number will be reused by
|
||||
// something else. Although this DomainSocket object continues to hold
|
||||
// the old file descriptor number (it's a final field), we never use it
|
||||
// again because this DomainSocket is closed.
|
||||
close0(fd);
|
||||
if (interrupted) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
private native static void sendFileDescriptors0(int fd,
|
||||
FileDescriptor descriptors[],
|
||||
byte jbuf[], int offset, int length) throws IOException;
|
||||
|
||||
/**
|
||||
* Send some FileDescriptor objects to the process on the other side of this
|
||||
* socket.
|
||||
*
|
||||
* @param descriptors The file descriptors to send.
|
||||
* @param jbuf Some bytes to send. You must send at least
|
||||
* one byte.
|
||||
* @param offset The offset in the jbuf array to start at.
|
||||
* @param length Length of the jbuf array to use.
|
||||
*/
|
||||
public void sendFileDescriptors(FileDescriptor descriptors[],
|
||||
byte jbuf[], int offset, int length) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
sendFileDescriptors0(fd, descriptors, jbuf, offset, length);
|
||||
exc = false;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private static native int receiveFileDescriptors0(int fd,
|
||||
FileDescriptor[] descriptors,
|
||||
byte jbuf[], int offset, int length) throws IOException;
|
||||
|
||||
/**
|
||||
* Receive some FileDescriptor objects from the process on the other side of
|
||||
* this socket.
|
||||
*
|
||||
* @param descriptors (output parameter) Array of FileDescriptors.
|
||||
* We will fill as many slots as possible with file
|
||||
* descriptors passed from the remote process. The
|
||||
* other slots will contain NULL.
|
||||
* @param jbuf (output parameter) Buffer to read into.
|
||||
* The UNIX domain sockets API requires you to read
|
||||
* at least one byte from the remote process, even
|
||||
* if all you care about is the file descriptors
|
||||
* you will receive.
|
||||
* @param offset Offset into the byte buffer to load data
|
||||
* @param length Length of the byte buffer to use for data
|
||||
*
|
||||
* @return The number of bytes read. This will be -1 if we
|
||||
* reached EOF (similar to SocketInputStream);
|
||||
* otherwise, it will be positive.
|
||||
* @throws IOException if there was an I/O error.
|
||||
*/
|
||||
public int receiveFileDescriptors(FileDescriptor[] descriptors,
|
||||
byte jbuf[], int offset, int length) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
int nBytes = receiveFileDescriptors0(fd, descriptors, jbuf, offset, length);
|
||||
exc = false;
|
||||
return nBytes;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Receive some FileDescriptor objects from the process on the other side of
|
||||
* this socket, and wrap them in FileInputStream objects.
|
||||
*
|
||||
* See {@link DomainSocket#recvFileInputStreams(ByteBuffer)}
|
||||
*/
|
||||
public int recvFileInputStreams(FileInputStream[] streams, byte buf[],
|
||||
int offset, int length) throws IOException {
|
||||
FileDescriptor descriptors[] = new FileDescriptor[streams.length];
|
||||
boolean success = false;
|
||||
for (int i = 0; i < streams.length; i++) {
|
||||
streams[i] = null;
|
||||
}
|
||||
status.reference();
|
||||
try {
|
||||
int ret = receiveFileDescriptors0(fd, descriptors, buf, offset, length);
|
||||
for (int i = 0, j = 0; i < descriptors.length; i++) {
|
||||
if (descriptors[i] != null) {
|
||||
streams[j++] = new FileInputStream(descriptors[i]);
|
||||
descriptors[i] = null;
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
for (int i = 0; i < descriptors.length; i++) {
|
||||
if (descriptors[i] != null) {
|
||||
try {
|
||||
closeFileDescriptor0(descriptors[i]);
|
||||
} catch (Throwable t) {
|
||||
LOG.warn(t);
|
||||
}
|
||||
} else if (streams[i] != null) {
|
||||
try {
|
||||
streams[i].close();
|
||||
} catch (Throwable t) {
|
||||
LOG.warn(t);
|
||||
} finally {
|
||||
streams[i] = null; }
|
||||
}
|
||||
}
|
||||
}
|
||||
status.unreference(!success);
|
||||
}
|
||||
}
|
||||
|
||||
private native static int readArray0(int fd, byte b[], int off, int len)
|
||||
throws IOException;
|
||||
|
||||
private native static int available0(int fd) throws IOException;
|
||||
|
||||
private static native void write0(int fd, int b) throws IOException;
|
||||
|
||||
private static native void writeArray0(int fd, byte b[], int offset, int length)
|
||||
throws IOException;
|
||||
|
||||
private native static int readByteBufferDirect0(int fd, ByteBuffer dst,
|
||||
int position, int remaining) throws IOException;
|
||||
|
||||
/**
|
||||
* Input stream for UNIX domain sockets.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
public class DomainInputStream extends InputStream {
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
byte b[] = new byte[1];
|
||||
int ret = DomainSocket.readArray0(DomainSocket.this.fd, b, 0, 1);
|
||||
exc = false;
|
||||
return (ret >= 0) ? b[0] : -1;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte b[], int off, int len) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
int nRead = DomainSocket.readArray0(DomainSocket.this.fd, b, off, len);
|
||||
exc = false;
|
||||
return nRead;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
int nAvailable = DomainSocket.available0(DomainSocket.this.fd);
|
||||
exc = false;
|
||||
return nAvailable;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
DomainSocket.this.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Output stream for UNIX domain sockets.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
public class DomainOutputStream extends OutputStream {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
DomainSocket.this.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(int val) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
byte b[] = new byte[1];
|
||||
b[0] = (byte)val;
|
||||
DomainSocket.writeArray0(DomainSocket.this.fd, b, 0, 1);
|
||||
exc = false;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] b, int off, int len) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
DomainSocket.writeArray0(DomainSocket.this.fd, b, off, len);
|
||||
exc = false;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
public class DomainChannel implements ReadableByteChannel {
|
||||
@Override
|
||||
public boolean isOpen() {
|
||||
return DomainSocket.this.isOpen();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
DomainSocket.this.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(ByteBuffer dst) throws IOException {
|
||||
status.reference();
|
||||
boolean exc = true;
|
||||
try {
|
||||
int nread = 0;
|
||||
if (dst.isDirect()) {
|
||||
nread = DomainSocket.readByteBufferDirect0(DomainSocket.this.fd,
|
||||
dst, dst.position(), dst.remaining());
|
||||
} else if (dst.hasArray()) {
|
||||
nread = DomainSocket.readArray0(DomainSocket.this.fd,
|
||||
dst.array(), dst.position() + dst.arrayOffset(),
|
||||
dst.remaining());
|
||||
} else {
|
||||
throw new AssertionError("we don't support " +
|
||||
"using ByteBuffers that aren't either direct or backed by " +
|
||||
"arrays");
|
||||
}
|
||||
if (nread > 0) {
|
||||
dst.position(dst.position() + nread);
|
||||
}
|
||||
exc = false;
|
||||
return nread;
|
||||
} finally {
|
||||
status.unreference(exc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("DomainSocket(fd=%d,path=%s)", fd, path);
|
||||
}
|
||||
}
|
|
@ -104,7 +104,7 @@ public class DataChecksum implements Checksum {
|
|||
( (bytes[offset+2] & 0xff) << 16 ) |
|
||||
( (bytes[offset+3] & 0xff) << 8 ) |
|
||||
( (bytes[offset+4] & 0xff) );
|
||||
return newDataChecksum( Type.valueOf(bytes[0]), bytesPerChecksum );
|
||||
return newDataChecksum( Type.valueOf(bytes[offset]), bytesPerChecksum );
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,109 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "exception.h"
|
||||
|
||||
#include <jni.h>
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <string.h>
|
||||
|
||||
jthrowable newExceptionV(JNIEnv* env, const char *name,
|
||||
const char *fmt, va_list ap)
|
||||
{
|
||||
int need;
|
||||
char buf[1], *msg = NULL;
|
||||
va_list ap2;
|
||||
jstring jstr = NULL;
|
||||
jthrowable jthr;
|
||||
jclass clazz;
|
||||
jmethodID excCtor;
|
||||
|
||||
va_copy(ap2, ap);
|
||||
clazz = (*env)->FindClass(env, name);
|
||||
if (!clazz) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
excCtor = (*env)->GetMethodID(env,
|
||||
clazz, "<init>", "(Ljava/lang/String;)V");
|
||||
if (!excCtor) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
need = vsnprintf(buf, sizeof(buf), fmt, ap);
|
||||
if (need < 0) {
|
||||
fmt = "vsnprintf error";
|
||||
need = strlen(fmt);
|
||||
}
|
||||
msg = malloc(need + 1);
|
||||
vsnprintf(msg, need + 1, fmt, ap2);
|
||||
jstr = (*env)->NewStringUTF(env, msg);
|
||||
if (!jstr) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
jthr = (*env)->NewObject(env, clazz, excCtor, jstr);
|
||||
if (!jthr) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
|
||||
done:
|
||||
free(msg);
|
||||
va_end(ap2);
|
||||
(*env)->DeleteLocalRef(env, jstr);
|
||||
return jthr;
|
||||
}
|
||||
|
||||
jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
|
||||
{
|
||||
va_list ap;
|
||||
jthrowable jthr;
|
||||
|
||||
va_start(ap, fmt);
|
||||
jthr = newExceptionV(env, name, fmt, ap);
|
||||
va_end(ap);
|
||||
return jthr;
|
||||
}
|
||||
|
||||
jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
|
||||
{
|
||||
va_list ap;
|
||||
jthrowable jthr;
|
||||
|
||||
va_start(ap, fmt);
|
||||
jthr = newExceptionV(env, "java/lang/RuntimeException", fmt, ap);
|
||||
va_end(ap);
|
||||
return jthr;
|
||||
}
|
||||
|
||||
jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
|
||||
{
|
||||
va_list ap;
|
||||
jthrowable jthr;
|
||||
|
||||
va_start(ap, fmt);
|
||||
jthr = newExceptionV(env, "java/io/IOException", fmt, ap);
|
||||
va_end(ap);
|
||||
return jthr;
|
||||
}
|
|
@ -0,0 +1,82 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
#ifndef HADOOP_MAIN_NATIVE_SRC_EXCEPTION_H
|
||||
#define HADOOP_MAIN_NATIVE_SRC_EXCEPTION_H
|
||||
|
||||
#include <jni.h> /* for jthrowable */
|
||||
#include <stdarg.h> /* for va_list */
|
||||
|
||||
/**
|
||||
* Create a new Exception.
|
||||
*
|
||||
* No exceptions will be pending on return.
|
||||
*
|
||||
* @param env The JNI environment
|
||||
* @param name full name of the Java exception class
|
||||
* @param fmt printf-style format string
|
||||
* @param ap printf-style arguments
|
||||
*
|
||||
* @return The RuntimeException
|
||||
*/
|
||||
jthrowable newExceptionV(JNIEnv* env, const char *name,
|
||||
const char *fmt, va_list ap);
|
||||
|
||||
/**
|
||||
* Create a new Exception.
|
||||
*
|
||||
* No exceptions will be pending on return.
|
||||
*
|
||||
* @param env The JNI environment
|
||||
* @param name full name of the Java exception class
|
||||
* @param fmt printf-style format string
|
||||
* @param ... printf-style arguments
|
||||
*
|
||||
* @return The RuntimeException
|
||||
*/
|
||||
jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
|
||||
__attribute__((format(printf, 3, 4)));
|
||||
|
||||
/**
|
||||
* Create a new RuntimeException.
|
||||
*
|
||||
* No exceptions will be pending on return.
|
||||
*
|
||||
* @param env The JNI environment
|
||||
* @param fmt printf-style format string
|
||||
* @param ... printf-style arguments
|
||||
*
|
||||
* @return The RuntimeException
|
||||
*/
|
||||
jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
|
||||
__attribute__((format(printf, 2, 3)));
|
||||
|
||||
/**
|
||||
* Create a new IOException.
|
||||
*
|
||||
* No exceptions will be pending on return.
|
||||
*
|
||||
* @param env The JNI environment
|
||||
* @param fmt printf-style format string
|
||||
* @param ... printf-style arguments
|
||||
*
|
||||
* @return The IOException, or another exception if we failed
|
||||
* to create the NativeIOException.
|
||||
*/
|
||||
jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
|
||||
__attribute__((format(printf, 2, 3)));
|
||||
|
||||
#endif
|
|
@ -0,0 +1,944 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#define _GNU_SOURCE
|
||||
|
||||
#include "exception.h"
|
||||
#include "org/apache/hadoop/io/nativeio/file_descriptor.h"
|
||||
#include "org_apache_hadoop.h"
|
||||
#include "org_apache_hadoop_net_unix_DomainSocket.h"
|
||||
|
||||
#include <errno.h>
|
||||
#include <fcntl.h>
|
||||
#include <inttypes.h>
|
||||
#include <jni.h>
|
||||
#include <limits.h>
|
||||
#include <stdint.h>
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <sys/ioctl.h> /* for FIONREAD */
|
||||
#include <sys/socket.h>
|
||||
#include <sys/stat.h>
|
||||
#include <sys/types.h>
|
||||
#include <sys/un.h>
|
||||
#include <unistd.h>
|
||||
|
||||
#define SEND_BUFFER_SIZE org_apache_hadoop_net_unix_DomainSocket_SEND_BUFFER_SIZE
|
||||
#define RECEIVE_BUFFER_SIZE org_apache_hadoop_net_unix_DomainSocket_RECEIVE_BUFFER_SIZE
|
||||
#define SEND_TIMEOUT org_apache_hadoop_net_unix_DomainSocket_SEND_TIMEOUT
|
||||
#define RECEIVE_TIMEOUT org_apache_hadoop_net_unix_DomainSocket_RECEIVE_TIMEOUT
|
||||
|
||||
#define DEFAULT_RECEIVE_TIMEOUT 120000
|
||||
#define DEFAULT_SEND_TIMEOUT 120000
|
||||
#define LISTEN_BACKLOG 128
|
||||
|
||||
/**
|
||||
* Can't pass more than this number of file descriptors in a single message.
|
||||
*/
|
||||
#define MAX_PASSED_FDS 16
|
||||
|
||||
static jthrowable setAttribute0(JNIEnv *env, jint fd, jint type, jint val);
|
||||
|
||||
/**
|
||||
* Convert an errno to a socket exception name.
|
||||
*
|
||||
* Note: we assume that all of these exceptions have a one-argument constructor
|
||||
* that takes a string.
|
||||
*
|
||||
* @return The exception class name
|
||||
*/
|
||||
static const char *errnoToSocketExceptionName(int errnum)
|
||||
{
|
||||
switch (errnum) {
|
||||
case EAGAIN:
|
||||
/* accept(2) returns EAGAIN when a socket timeout has been set, and that
|
||||
* timeout elapses without an incoming connection. This error code is also
|
||||
* used in non-blocking I/O, but we don't support that. */
|
||||
case ETIMEDOUT:
|
||||
return "java/net/SocketTimeoutException";
|
||||
case EHOSTDOWN:
|
||||
case EHOSTUNREACH:
|
||||
case ECONNREFUSED:
|
||||
return "java/net/NoRouteToHostException";
|
||||
case ENOTSUP:
|
||||
return "java/lang/UnsupportedOperationException";
|
||||
default:
|
||||
return "java/net/SocketException";
|
||||
}
|
||||
}
|
||||
|
||||
static jthrowable newSocketException(JNIEnv *env, int errnum,
|
||||
const char *fmt, ...)
|
||||
__attribute__((format(printf, 3, 4)));
|
||||
|
||||
static jthrowable newSocketException(JNIEnv *env, int errnum,
|
||||
const char *fmt, ...)
|
||||
{
|
||||
va_list ap;
|
||||
jthrowable jthr;
|
||||
|
||||
va_start(ap, fmt);
|
||||
jthr = newExceptionV(env, errnoToSocketExceptionName(errnum), fmt, ap);
|
||||
va_end(ap);
|
||||
return jthr;
|
||||
}
|
||||
|
||||
static const char* terror(int errnum)
|
||||
{
|
||||
if ((errnum < 0) || (errnum >= sys_nerr)) {
|
||||
return "unknown error.";
|
||||
}
|
||||
return sys_errlist[errnum];
|
||||
}
|
||||
|
||||
/**
|
||||
* Flexible buffer that will try to fit data on the stack, and fall back
|
||||
* to the heap if necessary.
|
||||
*/
|
||||
struct flexibleBuffer {
|
||||
int8_t *curBuf;
|
||||
int8_t *allocBuf;
|
||||
int8_t stackBuf[8196];
|
||||
};
|
||||
|
||||
static jthrowable flexBufInit(JNIEnv *env, struct flexibleBuffer *flexBuf, jint length)
|
||||
{
|
||||
flexBuf->curBuf = flexBuf->allocBuf = NULL;
|
||||
if (length < sizeof(flexBuf->stackBuf)) {
|
||||
flexBuf->curBuf = flexBuf->stackBuf;
|
||||
return NULL;
|
||||
}
|
||||
flexBuf->allocBuf = malloc(length);
|
||||
if (!flexBuf->allocBuf) {
|
||||
return newException(env, "java/lang/OutOfMemoryError",
|
||||
"OOM allocating space for %d bytes of data.", length);
|
||||
}
|
||||
flexBuf->curBuf = flexBuf->allocBuf;
|
||||
return NULL;
|
||||
}
|
||||
|
||||
static void flexBufFree(struct flexibleBuffer *flexBuf)
|
||||
{
|
||||
free(flexBuf->allocBuf);
|
||||
}
|
||||
|
||||
static jthrowable setup(JNIEnv *env, int *ofd, jobject jpath, int doConnect)
|
||||
{
|
||||
const char *cpath = NULL;
|
||||
struct sockaddr_un addr;
|
||||
jthrowable jthr = NULL;
|
||||
int fd = -1, ret;
|
||||
|
||||
fd = socket(PF_UNIX, SOCK_STREAM, 0);
|
||||
if (fd < 0) {
|
||||
ret = errno;
|
||||
jthr = newSocketException(env, ret,
|
||||
"error creating UNIX domain socket with SOCK_STREAM: %s",
|
||||
terror(ret));
|
||||
goto done;
|
||||
}
|
||||
memset(&addr, 0, sizeof(&addr));
|
||||
addr.sun_family = AF_UNIX;
|
||||
cpath = (*env)->GetStringUTFChars(env, jpath, NULL);
|
||||
if (!cpath) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
ret = snprintf(addr.sun_path, sizeof(addr.sun_path),
|
||||
"%s", cpath);
|
||||
if (ret < 0) {
|
||||
ret = errno;
|
||||
jthr = newSocketException(env, EIO,
|
||||
"error computing UNIX domain socket path: error %d (%s)",
|
||||
ret, terror(ret));
|
||||
goto done;
|
||||
}
|
||||
if (ret >= sizeof(addr.sun_path)) {
|
||||
jthr = newSocketException(env, ENAMETOOLONG,
|
||||
"error computing UNIX domain socket path: path too long. "
|
||||
"The longest UNIX domain socket path possible on this host "
|
||||
"is %zd bytes.", sizeof(addr.sun_path) - 1);
|
||||
goto done;
|
||||
}
|
||||
if (doConnect) {
|
||||
RETRY_ON_EINTR(ret, connect(fd,
|
||||
(struct sockaddr*)&addr, sizeof(addr)));
|
||||
if (ret < 0) {
|
||||
ret = errno;
|
||||
jthr = newException(env, "java/net/ConnectException",
|
||||
"connect(2) error: %s when trying to connect to '%s'",
|
||||
terror(ret), addr.sun_path);
|
||||
goto done;
|
||||
}
|
||||
} else {
|
||||
RETRY_ON_EINTR(ret, unlink(addr.sun_path));
|
||||
RETRY_ON_EINTR(ret, bind(fd, (struct sockaddr*)&addr, sizeof(addr)));
|
||||
if (ret < 0) {
|
||||
ret = errno;
|
||||
jthr = newException(env, "java/net/BindException",
|
||||
"bind(2) error: %s when trying to bind to '%s'",
|
||||
terror(ret), addr.sun_path);
|
||||
goto done;
|
||||
}
|
||||
/* We need to make the socket readable and writable for all users in the
|
||||
* system.
|
||||
*
|
||||
* If the system administrator doesn't want the socket to be accessible to
|
||||
* all users, he can simply adjust the +x permissions on one of the socket's
|
||||
* parent directories.
|
||||
*
|
||||
* See HDFS-4485 for more discussion.
|
||||
*/
|
||||
if (chmod(addr.sun_path, 0666)) {
|
||||
ret = errno;
|
||||
jthr = newException(env, "java/net/BindException",
|
||||
"chmod(%s, 0666) failed: %s", addr.sun_path, terror(ret));
|
||||
goto done;
|
||||
}
|
||||
if (listen(fd, LISTEN_BACKLOG) < 0) {
|
||||
ret = errno;
|
||||
jthr = newException(env, "java/net/BindException",
|
||||
"listen(2) error: %s when trying to listen to '%s'",
|
||||
terror(ret), addr.sun_path);
|
||||
goto done;
|
||||
}
|
||||
}
|
||||
|
||||
done:
|
||||
if (cpath) {
|
||||
(*env)->ReleaseStringUTFChars(env, jpath, cpath);
|
||||
}
|
||||
if (jthr) {
|
||||
if (fd > 0) {
|
||||
RETRY_ON_EINTR(ret, close(fd));
|
||||
fd = -1;
|
||||
}
|
||||
} else {
|
||||
*ofd = fd;
|
||||
}
|
||||
return jthr;
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_anchorNative(
|
||||
JNIEnv *env, jclass clazz)
|
||||
{
|
||||
fd_init(env); // for fd_get, fd_create, etc.
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_validateSocketPathSecurity0(
|
||||
JNIEnv *env, jclass clazz, jobject jstr, jint skipComponents)
|
||||
{
|
||||
jint utfLength;
|
||||
char path[PATH_MAX], check[PATH_MAX], *token, *rest;
|
||||
struct stat st;
|
||||
int ret, mode, strlenPath;
|
||||
uid_t uid;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
utfLength = (*env)->GetStringUTFLength(env, jstr);
|
||||
if (utfLength > sizeof(path)) {
|
||||
jthr = newIOException(env, "path is too long! We expected a path "
|
||||
"no longer than %zd UTF-8 bytes.", sizeof(path));
|
||||
goto done;
|
||||
}
|
||||
(*env)->GetStringUTFRegion(env, jstr, 0, utfLength, path);
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
if (jthr) {
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
uid = geteuid();
|
||||
strlenPath = strlen(path);
|
||||
if (strlenPath == 0) {
|
||||
jthr = newIOException(env, "socket path is empty.");
|
||||
goto done;
|
||||
}
|
||||
if (path[strlenPath - 1] == '/') {
|
||||
/* It makes no sense to have a socket path that ends in a slash, since
|
||||
* sockets are not directories. */
|
||||
jthr = newIOException(env, "bad socket path '%s'. The socket path "
|
||||
"must not end in a slash.", path);
|
||||
goto done;
|
||||
}
|
||||
// This loop iterates through all of the path components except for the very
|
||||
// last one. We don't validate the last component, since it's not supposed to
|
||||
// be a directory. (If it is a directory, we will fail to create the socket
|
||||
// later with EISDIR or similar.)
|
||||
for (check[0] = '/', check[1] = '\0', rest = path, token = "";
|
||||
token && rest[0];
|
||||
token = strtok_r(rest, "/", &rest)) {
|
||||
if (strcmp(check, "/") != 0) {
|
||||
// If the previous directory we checked was '/', we skip appending another
|
||||
// slash to the end because it would be unncessary. Otherwise we do it.
|
||||
strcat(check, "/");
|
||||
}
|
||||
// These strcats are safe because the length of 'check' is the same as the
|
||||
// length of 'path' and we never add more slashes than were in the original
|
||||
// path.
|
||||
strcat(check, token);
|
||||
if (skipComponents > 0) {
|
||||
skipComponents--;
|
||||
continue;
|
||||
}
|
||||
if (stat(check, &st) < 0) {
|
||||
ret = errno;
|
||||
jthr = newIOException(env, "failed to stat a path component: '%s'. "
|
||||
"error code %d (%s)", check, ret, terror(ret));
|
||||
goto done;
|
||||
}
|
||||
mode = st.st_mode & 0777;
|
||||
if (mode & 0002) {
|
||||
jthr = newIOException(env, "the path component: '%s' is "
|
||||
"world-writable. Its permissions are 0%03o. Please fix "
|
||||
"this or select a different socket path.", check, mode);
|
||||
goto done;
|
||||
}
|
||||
if ((mode & 0020) && (st.st_gid != 0)) {
|
||||
jthr = newIOException(env, "the path component: '%s' is "
|
||||
"group-writable, and the group is not root. Its permissions are "
|
||||
"0%03o, and it is owned by gid %d. Please fix this or "
|
||||
"select a different socket path.", check, mode, st.st_gid);
|
||||
goto done;
|
||||
}
|
||||
if ((mode & 0200) && (st.st_uid != 0) &&
|
||||
(st.st_uid != uid)) {
|
||||
jthr = newIOException(env, "the path component: '%s' is "
|
||||
"owned by a user who is not root and not you. Your effective user "
|
||||
"id is %d; the path is owned by user id %d, and its permissions are "
|
||||
"0%03o. Please fix this or select a different socket path.",
|
||||
check, uid, st.st_uid, mode);
|
||||
goto done;
|
||||
goto done;
|
||||
}
|
||||
}
|
||||
done:
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_bind0(
|
||||
JNIEnv *env, jclass clazz, jstring path)
|
||||
{
|
||||
int fd;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
jthr = setup(env, &fd, path, 0);
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
return fd;
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_accept0(
|
||||
JNIEnv *env, jclass clazz, jint fd)
|
||||
{
|
||||
int ret, newFd = -1;
|
||||
socklen_t slen;
|
||||
struct sockaddr_un remote;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
slen = sizeof(remote);
|
||||
do {
|
||||
newFd = accept(fd, (struct sockaddr*)&remote, &slen);
|
||||
} while ((newFd < 0) && (errno == EINTR));
|
||||
if (newFd < 0) {
|
||||
ret = errno;
|
||||
jthr = newSocketException(env, ret, "accept(2) error: %s", terror(ret));
|
||||
goto done;
|
||||
}
|
||||
|
||||
done:
|
||||
if (jthr) {
|
||||
if (newFd > 0) {
|
||||
RETRY_ON_EINTR(ret, close(newFd));
|
||||
newFd = -1;
|
||||
}
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
return newFd;
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_connect0(
|
||||
JNIEnv *env, jclass clazz, jstring path)
|
||||
{
|
||||
int ret, fd;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
jthr = setup(env, &fd, path, 1);
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
return -1;
|
||||
}
|
||||
if (((jthr = setAttribute0(env, fd, SEND_TIMEOUT, DEFAULT_SEND_TIMEOUT))) ||
|
||||
((jthr = setAttribute0(env, fd, RECEIVE_TIMEOUT, DEFAULT_RECEIVE_TIMEOUT)))) {
|
||||
RETRY_ON_EINTR(ret, close(fd));
|
||||
(*env)->Throw(env, jthr);
|
||||
return -1;
|
||||
}
|
||||
return fd;
|
||||
}
|
||||
|
||||
static void javaMillisToTimeVal(int javaMillis, struct timeval *tv)
|
||||
{
|
||||
tv->tv_sec = javaMillis / 1000;
|
||||
tv->tv_usec = (javaMillis - (tv->tv_sec * 1000)) * 1000;
|
||||
}
|
||||
|
||||
static jthrowable setAttribute0(JNIEnv *env, jint fd, jint type, jint val)
|
||||
{
|
||||
struct timeval tv;
|
||||
int ret, buf;
|
||||
|
||||
switch (type) {
|
||||
case SEND_BUFFER_SIZE:
|
||||
buf = val;
|
||||
if (setsockopt(fd, SOL_SOCKET, SO_SNDBUF, &buf, sizeof(buf))) {
|
||||
ret = errno;
|
||||
return newSocketException(env, ret,
|
||||
"setsockopt(SO_SNDBUF) error: %s", terror(ret));
|
||||
}
|
||||
return NULL;
|
||||
case RECEIVE_BUFFER_SIZE:
|
||||
buf = val;
|
||||
if (setsockopt(fd, SOL_SOCKET, SO_RCVBUF, &buf, sizeof(buf))) {
|
||||
ret = errno;
|
||||
return newSocketException(env, ret,
|
||||
"setsockopt(SO_RCVBUF) error: %s", terror(ret));
|
||||
}
|
||||
return NULL;
|
||||
case SEND_TIMEOUT:
|
||||
javaMillisToTimeVal(val, &tv);
|
||||
if (setsockopt(fd, SOL_SOCKET, SO_SNDTIMEO, (struct timeval *)&tv,
|
||||
sizeof(tv))) {
|
||||
ret = errno;
|
||||
return newSocketException(env, ret,
|
||||
"setsockopt(SO_SNDTIMEO) error: %s", terror(ret));
|
||||
}
|
||||
return NULL;
|
||||
case RECEIVE_TIMEOUT:
|
||||
javaMillisToTimeVal(val, &tv);
|
||||
if (setsockopt(fd, SOL_SOCKET, SO_RCVTIMEO, (struct timeval *)&tv,
|
||||
sizeof(tv))) {
|
||||
ret = errno;
|
||||
return newSocketException(env, ret,
|
||||
"setsockopt(SO_RCVTIMEO) error: %s", terror(ret));
|
||||
}
|
||||
return NULL;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return newRuntimeException(env, "Invalid attribute type %d.", type);
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_setAttribute0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jint type, jint val)
|
||||
{
|
||||
jthrowable jthr = setAttribute0(env, fd, type, val);
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
}
|
||||
|
||||
static jint getSockOptBufSizeToJavaBufSize(int size)
|
||||
{
|
||||
#ifdef __linux__
|
||||
// Linux always doubles the value that you set with setsockopt.
|
||||
// We cut it in half here so that programs can at least read back the same
|
||||
// value they set.
|
||||
size /= 2;
|
||||
#endif
|
||||
return size;
|
||||
}
|
||||
|
||||
static int timeValToJavaMillis(const struct timeval *tv)
|
||||
{
|
||||
return (tv->tv_sec * 1000) + (tv->tv_usec / 1000);
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_getAttribute0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jint type)
|
||||
{
|
||||
struct timeval tv;
|
||||
socklen_t len;
|
||||
int ret, rval = 0;
|
||||
|
||||
switch (type) {
|
||||
case SEND_BUFFER_SIZE:
|
||||
len = sizeof(rval);
|
||||
if (getsockopt(fd, SOL_SOCKET, SO_SNDBUF, &rval, &len)) {
|
||||
ret = errno;
|
||||
(*env)->Throw(env, newSocketException(env, ret,
|
||||
"getsockopt(SO_SNDBUF) error: %s", terror(ret)));
|
||||
return -1;
|
||||
}
|
||||
return getSockOptBufSizeToJavaBufSize(rval);
|
||||
case RECEIVE_BUFFER_SIZE:
|
||||
len = sizeof(rval);
|
||||
if (getsockopt(fd, SOL_SOCKET, SO_RCVBUF, &rval, &len)) {
|
||||
ret = errno;
|
||||
(*env)->Throw(env, newSocketException(env, ret,
|
||||
"getsockopt(SO_RCVBUF) error: %s", terror(ret)));
|
||||
return -1;
|
||||
}
|
||||
return getSockOptBufSizeToJavaBufSize(rval);
|
||||
case SEND_TIMEOUT:
|
||||
memset(&tv, 0, sizeof(tv));
|
||||
len = sizeof(struct timeval);
|
||||
if (getsockopt(fd, SOL_SOCKET, SO_SNDTIMEO, &tv, &len)) {
|
||||
ret = errno;
|
||||
(*env)->Throw(env, newSocketException(env, ret,
|
||||
"getsockopt(SO_SNDTIMEO) error: %s", terror(ret)));
|
||||
return -1;
|
||||
}
|
||||
return timeValToJavaMillis(&tv);
|
||||
case RECEIVE_TIMEOUT:
|
||||
memset(&tv, 0, sizeof(tv));
|
||||
len = sizeof(struct timeval);
|
||||
if (getsockopt(fd, SOL_SOCKET, SO_RCVTIMEO, &tv, &len)) {
|
||||
ret = errno;
|
||||
(*env)->Throw(env, newSocketException(env, ret,
|
||||
"getsockopt(SO_RCVTIMEO) error: %s", terror(ret)));
|
||||
return -1;
|
||||
}
|
||||
return timeValToJavaMillis(&tv);
|
||||
default:
|
||||
(*env)->Throw(env, newRuntimeException(env,
|
||||
"Invalid attribute type %d.", type));
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_close0(
|
||||
JNIEnv *env, jclass clazz, jint fd)
|
||||
{
|
||||
int ret;
|
||||
|
||||
RETRY_ON_EINTR(ret, close(fd));
|
||||
if (ret) {
|
||||
ret = errno;
|
||||
(*env)->Throw(env, newSocketException(env, ret,
|
||||
"close(2) error: %s", terror(ret)));
|
||||
}
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_closeFileDescriptor0(
|
||||
JNIEnv *env, jclass clazz, jobject jfd)
|
||||
{
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_close0(
|
||||
env, clazz, fd_get(env, jfd));
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_shutdown0(
|
||||
JNIEnv *env, jclass clazz, jint fd)
|
||||
{
|
||||
int ret;
|
||||
|
||||
RETRY_ON_EINTR(ret, shutdown(fd, SHUT_RDWR));
|
||||
if (ret) {
|
||||
ret = errno;
|
||||
(*env)->Throw(env, newSocketException(env, ret,
|
||||
"shutdown(2) error: %s", terror(ret)));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write an entire buffer to a file descriptor.
|
||||
*
|
||||
* @param env The JNI environment.
|
||||
* @param fd The fd to write to.
|
||||
* @param buf The buffer to write
|
||||
* @param amt The length of the buffer to write.
|
||||
* @return NULL on success; or the unraised exception representing
|
||||
* the problem.
|
||||
*/
|
||||
static jthrowable write_fully(JNIEnv *env, int fd, int8_t *buf, int amt)
|
||||
{
|
||||
int err, res;
|
||||
|
||||
while (amt > 0) {
|
||||
res = write(fd, buf, amt);
|
||||
if (res < 0) {
|
||||
err = errno;
|
||||
if (err == EINTR) {
|
||||
continue;
|
||||
}
|
||||
return newSocketException(env, err, "write(2) error: %s", terror(err));
|
||||
}
|
||||
amt -= res;
|
||||
buf += res;
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
/**
|
||||
* Our auxillary data setup.
|
||||
*
|
||||
* See man 3 cmsg for more information about auxillary socket data on UNIX.
|
||||
*
|
||||
* We use __attribute__((packed)) to ensure that the compiler doesn't insert any
|
||||
* padding between 'hdr' and 'fds'.
|
||||
* We use __attribute__((aligned(8)) to ensure that the compiler puts the start
|
||||
* of the structure at an address which is a multiple of 8. If we did not do
|
||||
* this, the attribute((packed)) would cause the compiler to generate a lot of
|
||||
* slow code for accessing unaligned memory.
|
||||
*/
|
||||
struct cmsghdr_with_fds {
|
||||
struct cmsghdr hdr;
|
||||
int fds[MAX_PASSED_FDS];
|
||||
} __attribute__((packed,aligned(8)));
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_sendFileDescriptors0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jobject jfds, jobject jbuf,
|
||||
jint offset, jint length)
|
||||
{
|
||||
struct iovec vec[1];
|
||||
struct flexibleBuffer flexBuf;
|
||||
struct cmsghdr_with_fds aux;
|
||||
jint jfdsLen;
|
||||
int i, ret = -1, auxLen;
|
||||
struct msghdr socketMsg;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
jthr = flexBufInit(env, &flexBuf, length);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
if (length <= 0) {
|
||||
jthr = newException(env, "java/lang/IllegalArgumentException",
|
||||
"You must write at least one byte.");
|
||||
goto done;
|
||||
}
|
||||
jfdsLen = (*env)->GetArrayLength(env, jfds);
|
||||
if (jfdsLen <= 0) {
|
||||
jthr = newException(env, "java/lang/IllegalArgumentException",
|
||||
"Called sendFileDescriptors with no file descriptors.");
|
||||
goto done;
|
||||
} else if (jfdsLen > MAX_PASSED_FDS) {
|
||||
jfdsLen = 0;
|
||||
jthr = newException(env, "java/lang/IllegalArgumentException",
|
||||
"Called sendFileDescriptors with an array of %d length. "
|
||||
"The maximum is %d.", jfdsLen, MAX_PASSED_FDS);
|
||||
goto done;
|
||||
}
|
||||
(*env)->GetByteArrayRegion(env, jbuf, offset, length, flexBuf.curBuf);
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
if (jthr) {
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
memset(&vec, 0, sizeof(vec));
|
||||
vec[0].iov_base = flexBuf.curBuf;
|
||||
vec[0].iov_len = length;
|
||||
auxLen = CMSG_LEN(jfdsLen * sizeof(int));
|
||||
memset(&aux, 0, auxLen);
|
||||
memset(&socketMsg, 0, sizeof(socketMsg));
|
||||
socketMsg.msg_iov = vec;
|
||||
socketMsg.msg_iovlen = 1;
|
||||
socketMsg.msg_control = &aux;
|
||||
socketMsg.msg_controllen = auxLen;
|
||||
aux.hdr.cmsg_len = auxLen;
|
||||
aux.hdr.cmsg_level = SOL_SOCKET;
|
||||
aux.hdr.cmsg_type = SCM_RIGHTS;
|
||||
for (i = 0; i < jfdsLen; i++) {
|
||||
jobject jfd = (*env)->GetObjectArrayElement(env, jfds, i);
|
||||
if (!jfd) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
if (jthr) {
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
jthr = newException(env, "java/lang/NullPointerException",
|
||||
"element %d of jfds was NULL.", i);
|
||||
goto done;
|
||||
}
|
||||
aux.fds[i] = fd_get(env, jfd);
|
||||
(*env)->DeleteLocalRef(env, jfd);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
}
|
||||
RETRY_ON_EINTR(ret, sendmsg(fd, &socketMsg, 0));
|
||||
if (ret < 0) {
|
||||
ret = errno;
|
||||
jthr = newSocketException(env, ret, "sendmsg(2) error: %s", terror(ret));
|
||||
goto done;
|
||||
}
|
||||
length -= ret;
|
||||
if (length > 0) {
|
||||
// Write the rest of the bytes we were asked to send.
|
||||
// This time, no fds will be attached.
|
||||
jthr = write_fully(env, fd, flexBuf.curBuf + ret, length);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
}
|
||||
|
||||
done:
|
||||
flexBufFree(&flexBuf);
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_receiveFileDescriptors0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jarray jfds, jarray jbuf,
|
||||
jint offset, jint length)
|
||||
{
|
||||
struct iovec vec[1];
|
||||
struct flexibleBuffer flexBuf;
|
||||
struct cmsghdr_with_fds aux;
|
||||
int i, jRecvFdsLen = 0, auxLen;
|
||||
jint jfdsLen = 0;
|
||||
struct msghdr socketMsg;
|
||||
ssize_t bytesRead = -1;
|
||||
jobject fdObj;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
jthr = flexBufInit(env, &flexBuf, length);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
if (length <= 0) {
|
||||
jthr = newRuntimeException(env, "You must read at least one byte.");
|
||||
goto done;
|
||||
}
|
||||
jfdsLen = (*env)->GetArrayLength(env, jfds);
|
||||
if (jfdsLen <= 0) {
|
||||
jthr = newException(env, "java/lang/IllegalArgumentException",
|
||||
"Called receiveFileDescriptors with an array of %d length. "
|
||||
"You must pass at least one fd.", jfdsLen);
|
||||
goto done;
|
||||
} else if (jfdsLen > MAX_PASSED_FDS) {
|
||||
jfdsLen = 0;
|
||||
jthr = newException(env, "java/lang/IllegalArgumentException",
|
||||
"Called receiveFileDescriptors with an array of %d length. "
|
||||
"The maximum is %d.", jfdsLen, MAX_PASSED_FDS);
|
||||
goto done;
|
||||
}
|
||||
for (i = 0; i < jfdsLen; i++) {
|
||||
(*env)->SetObjectArrayElement(env, jfds, i, NULL);
|
||||
}
|
||||
vec[0].iov_base = flexBuf.curBuf;
|
||||
vec[0].iov_len = length;
|
||||
auxLen = CMSG_LEN(jfdsLen * sizeof(int));
|
||||
memset(&aux, 0, auxLen);
|
||||
memset(&socketMsg, 0, auxLen);
|
||||
socketMsg.msg_iov = vec;
|
||||
socketMsg.msg_iovlen = 1;
|
||||
socketMsg.msg_control = &aux;
|
||||
socketMsg.msg_controllen = auxLen;
|
||||
aux.hdr.cmsg_len = auxLen;
|
||||
aux.hdr.cmsg_level = SOL_SOCKET;
|
||||
aux.hdr.cmsg_type = SCM_RIGHTS;
|
||||
RETRY_ON_EINTR(bytesRead, recvmsg(fd, &socketMsg, 0));
|
||||
if (bytesRead < 0) {
|
||||
int ret = errno;
|
||||
if (ret == ECONNABORTED) {
|
||||
// The remote peer disconnected on us. Treat this as an EOF.
|
||||
bytesRead = -1;
|
||||
goto done;
|
||||
}
|
||||
jthr = newSocketException(env, ret, "recvmsg(2) failed: %s",
|
||||
terror(ret));
|
||||
goto done;
|
||||
} else if (bytesRead == 0) {
|
||||
bytesRead = -1;
|
||||
goto done;
|
||||
}
|
||||
jRecvFdsLen = (aux.hdr.cmsg_len - sizeof(struct cmsghdr)) / sizeof(int);
|
||||
for (i = 0; i < jRecvFdsLen; i++) {
|
||||
fdObj = fd_create(env, aux.fds[i]);
|
||||
if (!fdObj) {
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
// Make this -1 so we don't attempt to close it twice in an error path.
|
||||
aux.fds[i] = -1;
|
||||
(*env)->SetObjectArrayElement(env, jfds, i, fdObj);
|
||||
// There is no point keeping around a local reference to the fdObj.
|
||||
// The array continues to reference it.
|
||||
(*env)->DeleteLocalRef(env, fdObj);
|
||||
}
|
||||
(*env)->SetByteArrayRegion(env, jbuf, offset, length, flexBuf.curBuf);
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
if (jthr) {
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
done:
|
||||
flexBufFree(&flexBuf);
|
||||
if (jthr) {
|
||||
// Free any FileDescriptor references we may have created,
|
||||
// or file descriptors we may have been passed.
|
||||
for (i = 0; i < jRecvFdsLen; i++) {
|
||||
if (aux.fds[i] >= 0) {
|
||||
RETRY_ON_EINTR(i, close(aux.fds[i]));
|
||||
aux.fds[i] = -1;
|
||||
}
|
||||
fdObj = (*env)->GetObjectArrayElement(env, jfds, i);
|
||||
if (fdObj) {
|
||||
int ret, afd = fd_get(env, fdObj);
|
||||
if (afd >= 0) {
|
||||
RETRY_ON_EINTR(ret, close(afd));
|
||||
}
|
||||
(*env)->SetObjectArrayElement(env, jfds, i, NULL);
|
||||
(*env)->DeleteLocalRef(env, fdObj);
|
||||
}
|
||||
}
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_readArray0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jarray b, jint offset, jint length)
|
||||
{
|
||||
int ret = -1;
|
||||
struct flexibleBuffer flexBuf;
|
||||
jthrowable jthr;
|
||||
|
||||
jthr = flexBufInit(env, &flexBuf, length);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
RETRY_ON_EINTR(ret, read(fd, flexBuf.curBuf, length));
|
||||
if (ret < 0) {
|
||||
ret = errno;
|
||||
if (ret == ECONNABORTED) {
|
||||
// The remote peer disconnected on us. Treat this as an EOF.
|
||||
ret = -1;
|
||||
goto done;
|
||||
}
|
||||
jthr = newSocketException(env, ret, "read(2) error: %s",
|
||||
terror(ret));
|
||||
goto done;
|
||||
}
|
||||
if (ret == 0) {
|
||||
goto done;
|
||||
}
|
||||
(*env)->SetByteArrayRegion(env, b, offset, ret, flexBuf.curBuf);
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
if (jthr) {
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
done:
|
||||
flexBufFree(&flexBuf);
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
return ret == 0 ? -1 : ret; // Java wants -1 on EOF
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_available0(
|
||||
JNIEnv *env, jclass clazz, jint fd)
|
||||
{
|
||||
int ret, avail = 0;
|
||||
jthrowable jthr = NULL;
|
||||
|
||||
RETRY_ON_EINTR(ret, ioctl(fd, FIONREAD, &avail));
|
||||
if (ret < 0) {
|
||||
ret = errno;
|
||||
jthr = newSocketException(env, ret,
|
||||
"ioctl(%d, FIONREAD) error: %s", fd, terror(ret));
|
||||
goto done;
|
||||
}
|
||||
done:
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
return avail;
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_writeArray0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jarray b, jint offset, jint length)
|
||||
{
|
||||
struct flexibleBuffer flexBuf;
|
||||
jthrowable jthr;
|
||||
|
||||
jthr = flexBufInit(env, &flexBuf, length);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
(*env)->GetByteArrayRegion(env, b, offset, length, flexBuf.curBuf);
|
||||
jthr = (*env)->ExceptionOccurred(env);
|
||||
if (jthr) {
|
||||
(*env)->ExceptionClear(env);
|
||||
goto done;
|
||||
}
|
||||
jthr = write_fully(env, fd, flexBuf.curBuf, length);
|
||||
if (jthr) {
|
||||
goto done;
|
||||
}
|
||||
|
||||
done:
|
||||
flexBufFree(&flexBuf);
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
}
|
||||
|
||||
JNIEXPORT jint JNICALL
|
||||
Java_org_apache_hadoop_net_unix_DomainSocket_readByteBufferDirect0(
|
||||
JNIEnv *env, jclass clazz, jint fd, jobject dst, jint position, jint remaining)
|
||||
{
|
||||
uint8_t *buf;
|
||||
jthrowable jthr = NULL;
|
||||
int res = -1;
|
||||
|
||||
buf = (*env)->GetDirectBufferAddress(env, dst);
|
||||
if (!buf) {
|
||||
jthr = newRuntimeException(env, "GetDirectBufferAddress failed.");
|
||||
goto done;
|
||||
}
|
||||
RETRY_ON_EINTR(res, read(fd, buf + position, remaining));
|
||||
if (res < 0) {
|
||||
res = errno;
|
||||
if (res != ECONNABORTED) {
|
||||
jthr = newSocketException(env, res, "read(2) error: %s",
|
||||
terror(res));
|
||||
goto done;
|
||||
} else {
|
||||
// The remote peer disconnected on us. Treat this as an EOF.
|
||||
res = -1;
|
||||
}
|
||||
}
|
||||
done:
|
||||
if (jthr) {
|
||||
(*env)->Throw(env, jthr);
|
||||
}
|
||||
return res;
|
||||
}
|
|
@ -180,6 +180,10 @@ static FARPROC WINAPI do_dlsym(JNIEnv *env, HMODULE handle, LPCSTR symbol) {
|
|||
THROW(env, "java/lang/InternalError", exception_msg); \
|
||||
}
|
||||
|
||||
#define RETRY_ON_EINTR(ret, expr) do { \
|
||||
ret = expr; \
|
||||
} while ((ret == -1) && (errno == EINTR));
|
||||
|
||||
#endif
|
||||
|
||||
//vim: sw=2: ts=2: et
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* 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.net.unix;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
|
||||
/**
|
||||
* Create a temporary directory in which sockets can be created.
|
||||
* When creating a UNIX domain socket, the name
|
||||
* must be fairly short (around 110 bytes on most platforms).
|
||||
*/
|
||||
public class TemporarySocketDirectory implements Closeable {
|
||||
private File dir;
|
||||
|
||||
public TemporarySocketDirectory() {
|
||||
String tmp = System.getProperty("java.io.tmpdir", "/tmp");
|
||||
dir = new File(tmp, "socks." + (System.currentTimeMillis() +
|
||||
"." + (new Random().nextInt())));
|
||||
dir.mkdirs();
|
||||
dir.setWritable(true, true);
|
||||
}
|
||||
|
||||
public File getDir() {
|
||||
return dir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (dir != null) {
|
||||
FileUtils.deleteDirectory(dir);
|
||||
dir = null;
|
||||
}
|
||||
}
|
||||
|
||||
protected void finalize() throws IOException {
|
||||
close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,706 @@
|
|||
/**
|
||||
* 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.net.unix;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileDescriptor;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.AsynchronousCloseException;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.commons.lang.exception.ExceptionUtils;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket.DomainChannel;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
|
||||
import com.google.common.io.Files;
|
||||
|
||||
public class TestDomainSocket {
|
||||
private static TemporarySocketDirectory sockDir;
|
||||
|
||||
@BeforeClass
|
||||
public static void init() {
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
DomainSocket.disableBindPathValidation();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() throws IOException {
|
||||
sockDir.close();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we can create a socket and close it, even if it hasn't been
|
||||
* opened.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testSocketCreateAndClose() throws IOException {
|
||||
DomainSocket serv = DomainSocket.bindAndListen(
|
||||
new File(sockDir.getDir(), "test_sock_create_and_close").
|
||||
getAbsolutePath());
|
||||
serv.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test DomainSocket path setting and getting.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testSocketPathSetGet() throws IOException {
|
||||
Assert.assertEquals("/var/run/hdfs/sock.100",
|
||||
DomainSocket.getEffectivePath("/var/run/hdfs/sock._PORT", 100));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we get a read result of -1 on EOF.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testSocketReadEof() throws Exception {
|
||||
final String TEST_PATH = new File(sockDir.getDir(),
|
||||
"testSocketReadEof").getAbsolutePath();
|
||||
final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
|
||||
ExecutorService exeServ = Executors.newSingleThreadExecutor();
|
||||
Callable<Void> callable = new Callable<Void>() {
|
||||
public Void call(){
|
||||
DomainSocket conn;
|
||||
try {
|
||||
conn = serv.accept();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("unexpected IOException", e);
|
||||
}
|
||||
byte buf[] = new byte[100];
|
||||
for (int i = 0; i < buf.length; i++) {
|
||||
buf[i] = 0;
|
||||
}
|
||||
try {
|
||||
Assert.assertEquals(-1, conn.getInputStream().read());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("unexpected IOException", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
Future<Void> future = exeServ.submit(callable);
|
||||
DomainSocket conn = DomainSocket.connect(serv.getPath());
|
||||
Thread.sleep(50);
|
||||
conn.close();
|
||||
serv.close();
|
||||
future.get(2, TimeUnit.MINUTES);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that if one thread is blocking in a read or write operation, another
|
||||
* thread can close the socket and stop the accept.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testSocketAcceptAndClose() throws Exception {
|
||||
final String TEST_PATH =
|
||||
new File(sockDir.getDir(), "test_sock_accept_and_close").getAbsolutePath();
|
||||
final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
|
||||
ExecutorService exeServ = Executors.newSingleThreadExecutor();
|
||||
Callable<Void> callable = new Callable<Void>() {
|
||||
public Void call(){
|
||||
try {
|
||||
serv.accept();
|
||||
throw new RuntimeException("expected the accept() to be " +
|
||||
"interrupted and fail");
|
||||
} catch (AsynchronousCloseException e) {
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("unexpected IOException", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
Future<Void> future = exeServ.submit(callable);
|
||||
Thread.sleep(500);
|
||||
serv.close();
|
||||
future.get(2, TimeUnit.MINUTES);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we get an AsynchronousCloseException when the DomainSocket
|
||||
* we're using is closed during a read or write operation.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
private void testAsyncCloseDuringIO(final boolean closeDuringWrite)
|
||||
throws Exception {
|
||||
final String TEST_PATH = new File(sockDir.getDir(),
|
||||
"testAsyncCloseDuringIO(" + closeDuringWrite + ")").getAbsolutePath();
|
||||
final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
|
||||
ExecutorService exeServ = Executors.newFixedThreadPool(2);
|
||||
Callable<Void> serverCallable = new Callable<Void>() {
|
||||
public Void call() {
|
||||
DomainSocket serverConn = null;
|
||||
try {
|
||||
serverConn = serv.accept();
|
||||
byte buf[] = new byte[100];
|
||||
for (int i = 0; i < buf.length; i++) {
|
||||
buf[i] = 0;
|
||||
}
|
||||
// The server just continues either writing or reading until someone
|
||||
// asynchronously closes the client's socket. At that point, all our
|
||||
// reads return EOF, and writes get a socket error.
|
||||
if (closeDuringWrite) {
|
||||
try {
|
||||
while (true) {
|
||||
serverConn.getOutputStream().write(buf);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
}
|
||||
} else {
|
||||
do { ; } while
|
||||
(serverConn.getInputStream().read(buf, 0, buf.length) != -1);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("unexpected IOException", e);
|
||||
} finally {
|
||||
IOUtils.cleanup(DomainSocket.LOG, serverConn);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
Future<Void> serverFuture = exeServ.submit(serverCallable);
|
||||
final DomainSocket clientConn = DomainSocket.connect(serv.getPath());
|
||||
Callable<Void> clientCallable = new Callable<Void>() {
|
||||
public Void call(){
|
||||
// The client writes or reads until another thread
|
||||
// asynchronously closes the socket. At that point, we should
|
||||
// get ClosedChannelException, or possibly its subclass
|
||||
// AsynchronousCloseException.
|
||||
byte buf[] = new byte[100];
|
||||
for (int i = 0; i < buf.length; i++) {
|
||||
buf[i] = 0;
|
||||
}
|
||||
try {
|
||||
if (closeDuringWrite) {
|
||||
while (true) {
|
||||
clientConn.getOutputStream().write(buf);
|
||||
}
|
||||
} else {
|
||||
while (true) {
|
||||
clientConn.getInputStream().read(buf, 0, buf.length);
|
||||
}
|
||||
}
|
||||
} catch (ClosedChannelException e) {
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("unexpected IOException", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
Future<Void> clientFuture = exeServ.submit(clientCallable);
|
||||
Thread.sleep(500);
|
||||
clientConn.close();
|
||||
serv.close();
|
||||
clientFuture.get(2, TimeUnit.MINUTES);
|
||||
serverFuture.get(2, TimeUnit.MINUTES);
|
||||
}
|
||||
|
||||
@Test(timeout=180000)
|
||||
public void testAsyncCloseDuringWrite() throws Exception {
|
||||
testAsyncCloseDuringIO(true);
|
||||
}
|
||||
|
||||
@Test(timeout=180000)
|
||||
public void testAsyncCloseDuringRead() throws Exception {
|
||||
testAsyncCloseDuringIO(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that attempting to connect to an invalid path doesn't work.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testInvalidOperations() throws IOException {
|
||||
try {
|
||||
DomainSocket.connect(
|
||||
new File(sockDir.getDir(), "test_sock_invalid_operation").
|
||||
getAbsolutePath());
|
||||
} catch (IOException e) {
|
||||
GenericTestUtils.assertExceptionContains("connect(2) error: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test setting some server options.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testServerOptions() throws Exception {
|
||||
final String TEST_PATH = new File(sockDir.getDir(),
|
||||
"test_sock_server_options").getAbsolutePath();
|
||||
DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
|
||||
try {
|
||||
// Let's set a new receive buffer size
|
||||
int bufSize = serv.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE);
|
||||
int newBufSize = bufSize / 2;
|
||||
serv.setAttribute(DomainSocket.RECEIVE_BUFFER_SIZE, newBufSize);
|
||||
int nextBufSize = serv.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE);
|
||||
Assert.assertEquals(newBufSize, nextBufSize);
|
||||
// Let's set a server timeout
|
||||
int newTimeout = 1000;
|
||||
serv.setAttribute(DomainSocket.RECEIVE_TIMEOUT, newTimeout);
|
||||
int nextTimeout = serv.getAttribute(DomainSocket.RECEIVE_TIMEOUT);
|
||||
Assert.assertEquals(newTimeout, nextTimeout);
|
||||
try {
|
||||
serv.accept();
|
||||
Assert.fail("expected the accept() to time out and fail");
|
||||
} catch (SocketTimeoutException e) {
|
||||
GenericTestUtils.assertExceptionContains("accept(2) error: ", e);
|
||||
}
|
||||
} finally {
|
||||
serv.close();
|
||||
Assert.assertFalse(serv.isOpen());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Throwable representing success.
|
||||
*
|
||||
* We can't use null to represent this, because you cannot insert null into
|
||||
* ArrayBlockingQueue.
|
||||
*/
|
||||
static class Success extends Throwable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
}
|
||||
|
||||
static interface WriteStrategy {
|
||||
/**
|
||||
* Initialize a WriteStrategy object from a Socket.
|
||||
*/
|
||||
public void init(DomainSocket s) throws IOException;
|
||||
|
||||
/**
|
||||
* Write some bytes.
|
||||
*/
|
||||
public void write(byte b[]) throws IOException;
|
||||
}
|
||||
|
||||
static class OutputStreamWriteStrategy implements WriteStrategy {
|
||||
private OutputStream outs = null;
|
||||
|
||||
public void init(DomainSocket s) throws IOException {
|
||||
outs = s.getOutputStream();
|
||||
}
|
||||
|
||||
public void write(byte b[]) throws IOException {
|
||||
outs.write(b);
|
||||
}
|
||||
}
|
||||
|
||||
abstract static class ReadStrategy {
|
||||
/**
|
||||
* Initialize a ReadStrategy object from a DomainSocket.
|
||||
*/
|
||||
public abstract void init(DomainSocket s) throws IOException;
|
||||
|
||||
/**
|
||||
* Read some bytes.
|
||||
*/
|
||||
public abstract int read(byte b[], int off, int length) throws IOException;
|
||||
|
||||
public void readFully(byte buf[], int off, int len) throws IOException {
|
||||
int toRead = len;
|
||||
while (toRead > 0) {
|
||||
int ret = read(buf, off, toRead);
|
||||
if (ret < 0) {
|
||||
throw new IOException( "Premature EOF from inputStream");
|
||||
}
|
||||
toRead -= ret;
|
||||
off += ret;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class InputStreamReadStrategy extends ReadStrategy {
|
||||
private InputStream ins = null;
|
||||
|
||||
@Override
|
||||
public void init(DomainSocket s) throws IOException {
|
||||
ins = s.getInputStream();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte b[], int off, int length) throws IOException {
|
||||
return ins.read(b, off, length);
|
||||
}
|
||||
}
|
||||
|
||||
static class DirectByteBufferReadStrategy extends ReadStrategy {
|
||||
private DomainChannel ch = null;
|
||||
|
||||
@Override
|
||||
public void init(DomainSocket s) throws IOException {
|
||||
ch = s.getChannel();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte b[], int off, int length) throws IOException {
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect(b.length);
|
||||
int nread = ch.read(buf);
|
||||
if (nread < 0) return nread;
|
||||
buf.flip();
|
||||
buf.get(b, off, nread);
|
||||
return nread;
|
||||
}
|
||||
}
|
||||
|
||||
static class ArrayBackedByteBufferReadStrategy extends ReadStrategy {
|
||||
private DomainChannel ch = null;
|
||||
|
||||
@Override
|
||||
public void init(DomainSocket s) throws IOException {
|
||||
ch = s.getChannel();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte b[], int off, int length) throws IOException {
|
||||
ByteBuffer buf = ByteBuffer.wrap(b);
|
||||
int nread = ch.read(buf);
|
||||
if (nread < 0) return nread;
|
||||
buf.flip();
|
||||
buf.get(b, off, nread);
|
||||
return nread;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test a simple client/server interaction.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
void testClientServer1(final Class<? extends WriteStrategy> writeStrategyClass,
|
||||
final Class<? extends ReadStrategy> readStrategyClass) throws Exception {
|
||||
final String TEST_PATH = new File(sockDir.getDir(),
|
||||
"test_sock_client_server1").getAbsolutePath();
|
||||
final byte clientMsg1[] = new byte[] { 0x1, 0x2, 0x3, 0x4, 0x5, 0x6 };
|
||||
final byte serverMsg1[] = new byte[] { 0x9, 0x8, 0x7, 0x6, 0x5 };
|
||||
final byte clientMsg2 = 0x45;
|
||||
final ArrayBlockingQueue<Throwable> threadResults =
|
||||
new ArrayBlockingQueue<Throwable>(2);
|
||||
final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
|
||||
Thread serverThread = new Thread() {
|
||||
public void run(){
|
||||
// Run server
|
||||
DomainSocket conn = null;
|
||||
try {
|
||||
conn = serv.accept();
|
||||
byte in1[] = new byte[clientMsg1.length];
|
||||
ReadStrategy reader = readStrategyClass.newInstance();
|
||||
reader.init(conn);
|
||||
reader.readFully(in1, 0, in1.length);
|
||||
Assert.assertTrue(Arrays.equals(clientMsg1, in1));
|
||||
WriteStrategy writer = writeStrategyClass.newInstance();
|
||||
writer.init(conn);
|
||||
writer.write(serverMsg1);
|
||||
InputStream connInputStream = conn.getInputStream();
|
||||
int in2 = connInputStream.read();
|
||||
Assert.assertEquals((int)clientMsg2, in2);
|
||||
conn.close();
|
||||
} catch (Throwable e) {
|
||||
threadResults.add(e);
|
||||
Assert.fail(e.getMessage());
|
||||
}
|
||||
threadResults.add(new Success());
|
||||
}
|
||||
};
|
||||
serverThread.start();
|
||||
|
||||
Thread clientThread = new Thread() {
|
||||
public void run(){
|
||||
try {
|
||||
DomainSocket client = DomainSocket.connect(TEST_PATH);
|
||||
WriteStrategy writer = writeStrategyClass.newInstance();
|
||||
writer.init(client);
|
||||
writer.write(clientMsg1);
|
||||
ReadStrategy reader = readStrategyClass.newInstance();
|
||||
reader.init(client);
|
||||
byte in1[] = new byte[serverMsg1.length];
|
||||
reader.readFully(in1, 0, in1.length);
|
||||
Assert.assertTrue(Arrays.equals(serverMsg1, in1));
|
||||
OutputStream clientOutputStream = client.getOutputStream();
|
||||
clientOutputStream.write(clientMsg2);
|
||||
client.close();
|
||||
} catch (Throwable e) {
|
||||
threadResults.add(e);
|
||||
}
|
||||
threadResults.add(new Success());
|
||||
}
|
||||
};
|
||||
clientThread.start();
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
Throwable t = threadResults.take();
|
||||
if (!(t instanceof Success)) {
|
||||
Assert.fail(t.getMessage() + ExceptionUtils.getStackTrace(t));
|
||||
}
|
||||
}
|
||||
serverThread.join(120000);
|
||||
clientThread.join(120000);
|
||||
serv.close();
|
||||
}
|
||||
|
||||
@Test(timeout=180000)
|
||||
public void testClientServerOutStreamInStream() throws Exception {
|
||||
testClientServer1(OutputStreamWriteStrategy.class,
|
||||
InputStreamReadStrategy.class);
|
||||
}
|
||||
|
||||
@Test(timeout=180000)
|
||||
public void testClientServerOutStreamInDbb() throws Exception {
|
||||
testClientServer1(OutputStreamWriteStrategy.class,
|
||||
DirectByteBufferReadStrategy.class);
|
||||
}
|
||||
|
||||
@Test(timeout=180000)
|
||||
public void testClientServerOutStreamInAbb() throws Exception {
|
||||
testClientServer1(OutputStreamWriteStrategy.class,
|
||||
ArrayBackedByteBufferReadStrategy.class);
|
||||
}
|
||||
|
||||
static private class PassedFile {
|
||||
private final int idx;
|
||||
private final byte[] contents;
|
||||
private FileInputStream fis;
|
||||
|
||||
public PassedFile(int idx) throws IOException {
|
||||
this.idx = idx;
|
||||
this.contents = new byte[] { (byte)(idx % 127) };
|
||||
Files.write(contents, new File(getPath()));
|
||||
this.fis = new FileInputStream(getPath());
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return new File(sockDir.getDir(), "passed_file" + idx).getAbsolutePath();
|
||||
}
|
||||
|
||||
public FileInputStream getInputStream() throws IOException {
|
||||
return fis;
|
||||
}
|
||||
|
||||
public void cleanup() throws IOException {
|
||||
new File(getPath()).delete();
|
||||
fis.close();
|
||||
}
|
||||
|
||||
public void checkInputStream(FileInputStream fis) throws IOException {
|
||||
byte buf[] = new byte[contents.length];
|
||||
IOUtils.readFully(fis, buf, 0, buf.length);
|
||||
Arrays.equals(contents, buf);
|
||||
}
|
||||
|
||||
protected void finalize() {
|
||||
try {
|
||||
cleanup();
|
||||
} catch(Throwable t) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test file descriptor passing.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testFdPassing() throws Exception {
|
||||
final String TEST_PATH =
|
||||
new File(sockDir.getDir(), "test_sock").getAbsolutePath();
|
||||
final byte clientMsg1[] = new byte[] { 0x11, 0x22, 0x33, 0x44, 0x55, 0x66 };
|
||||
final byte serverMsg1[] = new byte[] { 0x31, 0x30, 0x32, 0x34, 0x31, 0x33,
|
||||
0x44, 0x1, 0x1, 0x1, 0x1, 0x1 };
|
||||
final ArrayBlockingQueue<Throwable> threadResults =
|
||||
new ArrayBlockingQueue<Throwable>(2);
|
||||
final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
|
||||
final PassedFile passedFiles[] =
|
||||
new PassedFile[] { new PassedFile(1), new PassedFile(2) };
|
||||
final FileDescriptor passedFds[] = new FileDescriptor[passedFiles.length];
|
||||
for (int i = 0; i < passedFiles.length; i++) {
|
||||
passedFds[i] = passedFiles[i].getInputStream().getFD();
|
||||
}
|
||||
Thread serverThread = new Thread() {
|
||||
public void run(){
|
||||
// Run server
|
||||
DomainSocket conn = null;
|
||||
try {
|
||||
conn = serv.accept();
|
||||
byte in1[] = new byte[clientMsg1.length];
|
||||
InputStream connInputStream = conn.getInputStream();
|
||||
IOUtils.readFully(connInputStream, in1, 0, in1.length);
|
||||
Assert.assertTrue(Arrays.equals(clientMsg1, in1));
|
||||
DomainSocket domainConn = (DomainSocket)conn;
|
||||
domainConn.sendFileDescriptors(passedFds, serverMsg1, 0,
|
||||
serverMsg1.length);
|
||||
conn.close();
|
||||
} catch (Throwable e) {
|
||||
threadResults.add(e);
|
||||
Assert.fail(e.getMessage());
|
||||
}
|
||||
threadResults.add(new Success());
|
||||
}
|
||||
};
|
||||
serverThread.start();
|
||||
|
||||
Thread clientThread = new Thread() {
|
||||
public void run(){
|
||||
try {
|
||||
DomainSocket client = DomainSocket.connect(TEST_PATH);
|
||||
OutputStream clientOutputStream = client.getOutputStream();
|
||||
InputStream clientInputStream = client.getInputStream();
|
||||
clientOutputStream.write(clientMsg1);
|
||||
DomainSocket domainConn = (DomainSocket)client;
|
||||
byte in1[] = new byte[serverMsg1.length];
|
||||
FileInputStream recvFis[] = new FileInputStream[passedFds.length];
|
||||
int r = domainConn.
|
||||
recvFileInputStreams(recvFis, in1, 0, in1.length - 1);
|
||||
Assert.assertTrue(r > 0);
|
||||
IOUtils.readFully(clientInputStream, in1, r, in1.length - r);
|
||||
Assert.assertTrue(Arrays.equals(serverMsg1, in1));
|
||||
for (int i = 0; i < passedFds.length; i++) {
|
||||
Assert.assertNotNull(recvFis[i]);
|
||||
passedFiles[i].checkInputStream(recvFis[i]);
|
||||
}
|
||||
for (FileInputStream fis : recvFis) {
|
||||
fis.close();
|
||||
}
|
||||
client.close();
|
||||
} catch (Throwable e) {
|
||||
threadResults.add(e);
|
||||
}
|
||||
threadResults.add(new Success());
|
||||
}
|
||||
};
|
||||
clientThread.start();
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
Throwable t = threadResults.take();
|
||||
if (!(t instanceof Success)) {
|
||||
Assert.fail(t.getMessage() + ExceptionUtils.getStackTrace(t));
|
||||
}
|
||||
}
|
||||
serverThread.join(120000);
|
||||
clientThread.join(120000);
|
||||
serv.close();
|
||||
for (PassedFile pf : passedFiles) {
|
||||
pf.cleanup();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Run validateSocketPathSecurity
|
||||
*
|
||||
* @param str The path to validate
|
||||
* @param prefix A prefix to skip validation for
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void testValidateSocketPath(String str, String prefix)
|
||||
throws IOException {
|
||||
int skipComponents = 1;
|
||||
File prefixFile = new File(prefix);
|
||||
while (true) {
|
||||
prefixFile = prefixFile.getParentFile();
|
||||
if (prefixFile == null) {
|
||||
break;
|
||||
}
|
||||
skipComponents++;
|
||||
}
|
||||
DomainSocket.validateSocketPathSecurity0(str,
|
||||
skipComponents);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test file descriptor path security.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test(timeout=180000)
|
||||
public void testFdPassingPathSecurity() throws Exception {
|
||||
TemporarySocketDirectory tmp = new TemporarySocketDirectory();
|
||||
try {
|
||||
String prefix = tmp.getDir().getAbsolutePath();
|
||||
Shell.execCommand(new String [] {
|
||||
"mkdir", "-p", prefix + "/foo/bar/baz" });
|
||||
Shell.execCommand(new String [] {
|
||||
"chmod", "0700", prefix + "/foo/bar/baz" });
|
||||
Shell.execCommand(new String [] {
|
||||
"chmod", "0700", prefix + "/foo/bar" });
|
||||
Shell.execCommand(new String [] {
|
||||
"chmod", "0707", prefix + "/foo" });
|
||||
Shell.execCommand(new String [] {
|
||||
"mkdir", "-p", prefix + "/q1/q2" });
|
||||
Shell.execCommand(new String [] {
|
||||
"chmod", "0700", prefix + "/q1" });
|
||||
Shell.execCommand(new String [] {
|
||||
"chmod", "0700", prefix + "/q1/q2" });
|
||||
testValidateSocketPath(prefix + "/q1/q2", prefix);
|
||||
try {
|
||||
testValidateSocketPath(prefix + "/foo/bar/baz", prefix);
|
||||
} catch (IOException e) {
|
||||
GenericTestUtils.assertExceptionContains("/foo' is world-writable. " +
|
||||
"Its permissions are 0707. Please fix this or select a " +
|
||||
"different socket path.", e);
|
||||
}
|
||||
try {
|
||||
testValidateSocketPath(prefix + "/nope", prefix);
|
||||
} catch (IOException e) {
|
||||
GenericTestUtils.assertExceptionContains("failed to stat a path " +
|
||||
"component: ", e);
|
||||
}
|
||||
// Root should be secure
|
||||
DomainSocket.validateSocketPathSecurity0("/foo", 1);
|
||||
} finally {
|
||||
tmp.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -187,6 +187,9 @@ Trunk (Unreleased)
|
|||
HDFS-4633 TestDFSClientExcludedNodes fails sporadically if excluded nodes
|
||||
cache expires too quickly (Chris Nauroth via Sanjay)
|
||||
|
||||
HDFS-347. DFS read performance suboptimal when client co-located on nodes
|
||||
with data. (Colin Patrick McCabe via todd and atm)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -355,6 +358,62 @@ Trunk (Unreleased)
|
|||
HDFS-4674. TestBPOfferService fails on Windows due to failure parsing
|
||||
datanode data directory as URI. (Chris Nauroth via suresh)
|
||||
|
||||
BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
|
||||
|
||||
HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4354. Create DomainSocket and DomainPeer and associated unit tests.
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4356. BlockReaderLocal should use passed file descriptors rather than paths.
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4388. DomainSocket should throw AsynchronousCloseException when appropriate.
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4390. Bypass UNIX domain socket unit tests when they cannot be run.
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4400. DFSInputStream#getBlockReader: last retries should ignore the cache
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4401. Fix bug in DomainSocket path validation
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4402. Some small DomainSocket fixes: avoid findbugs warning, change
|
||||
log level, etc. (Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4418. increase default FileInputStreamCache size (todd)
|
||||
|
||||
HDFS-4416. Rename dfs.datanode.domain.socket.path to dfs.domain.socket.path
|
||||
(Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4417. Fix case where local reads get disabled incorrectly
|
||||
(Colin Patrick McCabe and todd via todd)
|
||||
|
||||
HDFS-4433. Make TestPeerCache not flaky (Colin Patrick McCabe via todd)
|
||||
|
||||
HDFS-4438. TestDomainSocket fails when system umask is set to 0002. (Colin
|
||||
Patrick McCabe via atm)
|
||||
|
||||
HDFS-4440. Avoid annoying log message when dfs.domain.socket.path is not
|
||||
set. (Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4473. Don't create domain socket unless we need it. (Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4485. DN should chmod socket path a+w. (Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4453. Make a simple doc to describe the usage and design of the
|
||||
shortcircuit read feature. (Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4496. DFSClient: don't create a domain socket unless we need it (Colin
|
||||
Patrick McCabe via todd)
|
||||
|
||||
HDFS-347: style cleanups (Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4538. Allow use of legacy blockreader (Colin Patrick McCabe via todd)
|
||||
|
||||
Release 2.0.5-beta - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -290,6 +290,14 @@
|
|||
<Method name="persistPaxosData" />
|
||||
<Bug pattern="OS_OPEN_STREAM" />
|
||||
</Match>
|
||||
|
||||
<!-- getShortCircuitFdsForRead is supposed to return open streams. -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl" />
|
||||
<Method name="getShortCircuitFdsForRead" />
|
||||
<Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
|
||||
</Match>
|
||||
|
||||
<!-- Don't complain about LocalDatanodeInfo's anonymous class -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.BlockReaderLocal$LocalDatanodeInfo$1" />
|
||||
|
|
|
@ -515,6 +515,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<excludes>
|
||||
<exclude>CHANGES.txt</exclude>
|
||||
<exclude>CHANGES.HDFS-1623.txt</exclude>
|
||||
<exclude>CHANGES.HDFS-347.txt</exclude>
|
||||
<exclude>.idea/**</exclude>
|
||||
<exclude>src/main/conf/*</exclude>
|
||||
<exclude>src/main/docs/**</exclude>
|
||||
|
|
|
@ -18,10 +18,8 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.Socket;
|
||||
|
||||
import org.apache.hadoop.fs.ByteBufferReadable;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
|
||||
/**
|
||||
* A BlockReader is responsible for reading a single block
|
||||
|
@ -43,7 +41,29 @@ public interface BlockReader extends ByteBufferReadable {
|
|||
*/
|
||||
long skip(long n) throws IOException;
|
||||
|
||||
void close() throws IOException;
|
||||
/**
|
||||
* Returns an estimate of the number of bytes that can be read
|
||||
* (or skipped over) from this input stream without performing
|
||||
* network I/O.
|
||||
*/
|
||||
int available() throws IOException;
|
||||
|
||||
/**
|
||||
* Close the block reader.
|
||||
*
|
||||
* @param peerCache The PeerCache to put the Peer we're using back
|
||||
* into, or null if we should simply close the Peer
|
||||
* we're using (along with its Socket).
|
||||
* Ignored by Readers that don't maintain Peers.
|
||||
* @param fisCache The FileInputStreamCache to put our FileInputStreams
|
||||
* back into, or null if we should simply close them.
|
||||
* Ignored by Readers that don't maintain
|
||||
* FileInputStreams.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
void close(PeerCache peerCache, FileInputStreamCache fisCache)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Read exactly the given amount of data, throwing an exception
|
||||
|
@ -60,20 +80,4 @@ public interface BlockReader extends ByteBufferReadable {
|
|||
* filled or the next call will return EOF.
|
||||
*/
|
||||
int readAll(byte[] buf, int offset, int len) throws IOException;
|
||||
|
||||
/**
|
||||
* Take the socket used to talk to the DN.
|
||||
*/
|
||||
Socket takeSocket();
|
||||
|
||||
/**
|
||||
* Whether the BlockReader has reached the end of its input stream
|
||||
* and successfully sent a status code back to the datanode.
|
||||
*/
|
||||
boolean hasSentStatusCode();
|
||||
|
||||
/**
|
||||
* @return a reference to the streams this block reader is using.
|
||||
*/
|
||||
IOStreamPair getStreams();
|
||||
}
|
||||
|
|
|
@ -17,20 +17,31 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
|
||||
|
@ -39,75 +50,182 @@ import org.apache.hadoop.security.token.Token;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BlockReaderFactory {
|
||||
/**
|
||||
* @see #newBlockReader(Conf, Socket, String, ExtendedBlock, Token, long, long, int, boolean, String)
|
||||
*/
|
||||
public static BlockReader newBlockReader(
|
||||
Configuration conf,
|
||||
Socket sock, String file,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len, DataEncryptionKey encryptionKey)
|
||||
throws IOException {
|
||||
int bufferSize = conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
|
||||
DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT);
|
||||
return newBlockReader(new Conf(conf),
|
||||
sock, file, block, blockToken, startOffset,
|
||||
len, bufferSize, true, "", encryptionKey, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new BlockReader specifically to satisfy a read.
|
||||
* This method also sends the OP_READ_BLOCK request.
|
||||
*
|
||||
* @param conf the DFSClient configuration
|
||||
* @param sock An established Socket to the DN. The BlockReader will not close it normally
|
||||
* @param file File location
|
||||
* @param block The block object
|
||||
* @param blockToken The block token for security
|
||||
* @param startOffset The read offset, relative to block head
|
||||
* @param len The number of bytes to read
|
||||
* @param len The number of bytes to read, or -1 to read as many as
|
||||
* possible.
|
||||
* @param bufferSize The IO buffer size (not the client buffer size)
|
||||
* Ignored except on the legacy BlockReader.
|
||||
* @param verifyChecksum Whether to verify checksum
|
||||
* @param clientName Client name
|
||||
* @return New BlockReader instance, or null on error.
|
||||
* @param clientName Client name. Used for log messages.
|
||||
* @param peer The peer
|
||||
* @param datanodeID The datanode that the Peer is connected to
|
||||
* @param domainSocketFactory The DomainSocketFactory to notify if the Peer
|
||||
* is a DomainPeer which turns out to be faulty.
|
||||
* If null, no factory will be notified in this
|
||||
* case.
|
||||
* @param allowShortCircuitLocalReads True if short-circuit local reads
|
||||
* should be allowed.
|
||||
* @return New BlockReader instance
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public static BlockReader newBlockReader(
|
||||
Conf conf,
|
||||
Socket sock, String file,
|
||||
Configuration conf,
|
||||
String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
int bufferSize, boolean verifyChecksum,
|
||||
boolean verifyChecksum,
|
||||
String clientName,
|
||||
DataEncryptionKey encryptionKey,
|
||||
IOStreamPair ioStreams)
|
||||
throws IOException {
|
||||
|
||||
if (conf.useLegacyBlockReader) {
|
||||
if (encryptionKey != null) {
|
||||
throw new RuntimeException("Encryption is not supported with the legacy block reader.");
|
||||
}
|
||||
return RemoteBlockReader.newBlockReader(
|
||||
sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, clientName);
|
||||
} else {
|
||||
if (ioStreams == null) {
|
||||
ioStreams = new IOStreamPair(NetUtils.getInputStream(sock),
|
||||
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT));
|
||||
if (encryptionKey != null) {
|
||||
IOStreamPair encryptedStreams =
|
||||
DataTransferEncryptor.getEncryptedStreams(
|
||||
ioStreams.out, ioStreams.in, encryptionKey);
|
||||
ioStreams = encryptedStreams;
|
||||
Peer peer,
|
||||
DatanodeID datanodeID,
|
||||
DomainSocketFactory domSockFactory,
|
||||
boolean allowShortCircuitLocalReads)
|
||||
throws IOException {
|
||||
peer.setReadTimeout(conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
||||
HdfsServerConstants.READ_TIMEOUT));
|
||||
peer.setWriteTimeout(HdfsServerConstants.WRITE_TIMEOUT);
|
||||
|
||||
if (peer.getDomainSocket() != null) {
|
||||
if (allowShortCircuitLocalReads &&
|
||||
(!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
|
||||
DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) {
|
||||
// If this is a domain socket, and short-circuit local reads are
|
||||
// enabled, try to set up a BlockReaderLocal.
|
||||
BlockReader reader = newShortCircuitBlockReader(conf, file,
|
||||
block, blockToken, startOffset, len, peer, datanodeID,
|
||||
domSockFactory, verifyChecksum);
|
||||
if (reader != null) {
|
||||
// One we've constructed the short-circuit block reader, we don't
|
||||
// need the socket any more. So let's return it to the cache.
|
||||
PeerCache peerCache = PeerCache.getInstance(
|
||||
conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT),
|
||||
conf.getLong(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT));
|
||||
peerCache.put(datanodeID, peer);
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
||||
// If this is a domain socket and we couldn't (or didn't want to) set
|
||||
// up a BlockReaderLocal, check that we are allowed to pass data traffic
|
||||
// over the socket before proceeding.
|
||||
if (!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
|
||||
DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
|
||||
throw new IOException("Because we can't do short-circuit access, " +
|
||||
"and data traffic over domain sockets is disabled, " +
|
||||
"we cannot use this socket to talk to " + datanodeID);
|
||||
}
|
||||
}
|
||||
|
||||
if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
|
||||
DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT)) {
|
||||
return RemoteBlockReader.newBlockReader(file,
|
||||
block, blockToken, startOffset, len,
|
||||
conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
|
||||
DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT),
|
||||
verifyChecksum, clientName, peer, datanodeID);
|
||||
} else {
|
||||
return RemoteBlockReader2.newBlockReader(
|
||||
sock, file, block, blockToken, startOffset, len, bufferSize,
|
||||
verifyChecksum, clientName, encryptionKey, ioStreams);
|
||||
file, block, blockToken, startOffset, len,
|
||||
verifyChecksum, clientName, peer, datanodeID);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a new short-circuit BlockReader.
|
||||
*
|
||||
* Here, we ask the DataNode to pass us file descriptors over our
|
||||
* DomainSocket. If the DataNode declines to do so, we'll return null here;
|
||||
* otherwise, we'll return the BlockReaderLocal. If the DataNode declines,
|
||||
* this function will inform the DomainSocketFactory that short-circuit local
|
||||
* reads are disabled for this DataNode, so that we don't ask again.
|
||||
*
|
||||
* @param conf the configuration.
|
||||
* @param file the file name. Used in log messages.
|
||||
* @param block The block object.
|
||||
* @param blockToken The block token for security.
|
||||
* @param startOffset The read offset, relative to block head.
|
||||
* @param len The number of bytes to read, or -1 to read
|
||||
* as many as possible.
|
||||
* @param peer The peer to use.
|
||||
* @param datanodeID The datanode that the Peer is connected to.
|
||||
* @param domSockFactory The DomainSocketFactory to notify if the Peer
|
||||
* is a DomainPeer which turns out to be faulty.
|
||||
* If null, no factory will be notified in this
|
||||
* case.
|
||||
* @param verifyChecksum True if we should verify the checksums.
|
||||
* Note: even if this is true, when
|
||||
* DFS_CLIENT_READ_CHECKSUM_SKIP_CHECKSUM_KEY is
|
||||
* set, we will skip checksums.
|
||||
*
|
||||
* @return The BlockReaderLocal, or null if the
|
||||
* DataNode declined to provide short-circuit
|
||||
* access.
|
||||
* @throws IOException If there was a communication error.
|
||||
*/
|
||||
private static BlockReaderLocal newShortCircuitBlockReader(
|
||||
Configuration conf, String file, ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken, long startOffset,
|
||||
long len, Peer peer, DatanodeID datanodeID,
|
||||
DomainSocketFactory domSockFactory, boolean verifyChecksum)
|
||||
throws IOException {
|
||||
final DataOutputStream out =
|
||||
new DataOutputStream(new BufferedOutputStream(
|
||||
peer.getOutputStream()));
|
||||
new Sender(out).requestShortCircuitFds(block, blockToken, 1);
|
||||
DataInputStream in =
|
||||
new DataInputStream(peer.getInputStream());
|
||||
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
|
||||
PBHelper.vintPrefixed(in));
|
||||
DomainSocket sock = peer.getDomainSocket();
|
||||
switch (resp.getStatus()) {
|
||||
case SUCCESS:
|
||||
BlockReaderLocal reader = null;
|
||||
byte buf[] = new byte[1];
|
||||
FileInputStream fis[] = new FileInputStream[2];
|
||||
sock.recvFileInputStreams(fis, buf, 0, buf.length);
|
||||
try {
|
||||
reader = new BlockReaderLocal(conf, file, block,
|
||||
startOffset, len, fis[0], fis[1], datanodeID, verifyChecksum);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
|
||||
}
|
||||
}
|
||||
return reader;
|
||||
case ERROR_UNSUPPORTED:
|
||||
if (!resp.hasShortCircuitAccessVersion()) {
|
||||
DFSClient.LOG.warn("short-circuit read access is disabled for " +
|
||||
"DataNode " + datanodeID + ". reason: " + resp.getMessage());
|
||||
domSockFactory.disableShortCircuitForPath(sock.getPath());
|
||||
} else {
|
||||
DFSClient.LOG.warn("short-circuit read access for the file " +
|
||||
file + " is disabled for DataNode " + datanodeID +
|
||||
". reason: " + resp.getMessage());
|
||||
}
|
||||
return null;
|
||||
case ERROR_ACCESS_TOKEN:
|
||||
String msg = "access control error while " +
|
||||
"attempting to set up short-circuit access to " +
|
||||
file + resp.getMessage();
|
||||
DFSClient.LOG.debug(msg);
|
||||
throw new InvalidBlockTokenException(msg);
|
||||
default:
|
||||
DFSClient.LOG.warn("error while attempting to set up short-circuit " +
|
||||
"access to " + file + ": " + resp.getMessage());
|
||||
domSockFactory.disableShortCircuitForPath(sock.getPath());
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* File name to print when accessing a block directly (from servlets)
|
||||
* @param s Address of the block location
|
||||
|
@ -119,4 +237,24 @@ public class BlockReaderFactory {
|
|||
final String poolId, final long blockId) {
|
||||
return s.toString() + ":" + poolId + ":" + blockId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get {@link BlockReaderLocalLegacy} for short circuited local reads.
|
||||
* This block reader implements the path-based style of local reads
|
||||
* first introduced in HDFS-2246.
|
||||
*/
|
||||
static BlockReader getLegacyBlockReaderLocal(UserGroupInformation ugi,
|
||||
Configuration conf, String src, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
|
||||
int socketTimeout, long offsetIntoBlock,
|
||||
boolean connectToDnViaHostname) throws InvalidToken, IOException {
|
||||
try {
|
||||
return BlockReaderLocalLegacy.newBlockReader(ugi, conf, src,
|
||||
blk, accessToken, chosenNode, socketTimeout, offsetIntoBlock,
|
||||
blk.getNumBytes() - offsetIntoBlock, connectToDnViaHostname);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException(InvalidToken.class,
|
||||
AccessControlException.class);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,33 +18,20 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.File;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.Socket;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.hdfs.util.DirectBufferPool;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
/**
|
||||
|
@ -56,84 +43,19 @@ import org.apache.hadoop.util.DataChecksum;
|
|||
* <ul>
|
||||
* <li>The client performing short circuit reads must be configured at the
|
||||
* datanode.</li>
|
||||
* <li>The client gets the path to the file where block is stored using
|
||||
* {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
|
||||
* RPC call</li>
|
||||
* <li>Client uses kerberos authentication to connect to the datanode over RPC,
|
||||
* if security is enabled.</li>
|
||||
* <li>The client gets the file descriptors for the metadata file and the data
|
||||
* file for the block using
|
||||
* {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
|
||||
* </li>
|
||||
* <li>The client reads the file descriptors.</li>
|
||||
* </ul>
|
||||
*/
|
||||
class BlockReaderLocal implements BlockReader {
|
||||
private static final Log LOG = LogFactory.getLog(DFSClient.class);
|
||||
|
||||
//Stores the cache and proxy for a local datanode.
|
||||
private static class LocalDatanodeInfo {
|
||||
private ClientDatanodeProtocol proxy = null;
|
||||
private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
|
||||
|
||||
LocalDatanodeInfo() {
|
||||
final int cacheSize = 10000;
|
||||
final float hashTableLoadFactor = 0.75f;
|
||||
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
|
||||
cache = Collections
|
||||
.synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
|
||||
hashTableCapacity, hashTableLoadFactor, true) {
|
||||
private static final long serialVersionUID = 1;
|
||||
|
||||
@Override
|
||||
protected boolean removeEldestEntry(
|
||||
Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
|
||||
return size() > cacheSize;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private synchronized ClientDatanodeProtocol getDatanodeProxy(
|
||||
UserGroupInformation ugi, final DatanodeInfo node,
|
||||
final Configuration conf, final int socketTimeout,
|
||||
final boolean connectToDnViaHostname) throws IOException {
|
||||
if (proxy == null) {
|
||||
try {
|
||||
proxy = ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
|
||||
@Override
|
||||
public ClientDatanodeProtocol run() throws Exception {
|
||||
return DFSUtil.createClientDatanodeProtocolProxy(node, conf,
|
||||
socketTimeout, connectToDnViaHostname);
|
||||
}
|
||||
});
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("encountered exception ", e);
|
||||
}
|
||||
}
|
||||
return proxy;
|
||||
}
|
||||
|
||||
private synchronized void resetDatanodeProxy() {
|
||||
if (null != proxy) {
|
||||
RPC.stopProxy(proxy);
|
||||
proxy = null;
|
||||
}
|
||||
}
|
||||
|
||||
private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
|
||||
return cache.get(b);
|
||||
}
|
||||
|
||||
private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
|
||||
cache.put(b, info);
|
||||
}
|
||||
|
||||
private void removeBlockLocalPathInfo(ExtendedBlock b) {
|
||||
cache.remove(b);
|
||||
}
|
||||
}
|
||||
|
||||
// Multiple datanodes could be running on the local machine. Store proxies in
|
||||
// a map keyed by the ipc port of the datanode.
|
||||
private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
|
||||
static final Log LOG = LogFactory.getLog(BlockReaderLocal.class);
|
||||
|
||||
private final FileInputStream dataIn; // reader for the data file
|
||||
private final FileInputStream checksumIn; // reader for the checksum file
|
||||
private final boolean verifyChecksum;
|
||||
|
||||
/**
|
||||
* Offset from the most recent chunk boundary at which the next read should
|
||||
|
@ -153,7 +75,6 @@ class BlockReaderLocal implements BlockReader {
|
|||
private ByteBuffer slowReadBuff = null;
|
||||
private ByteBuffer checksumBuff = null;
|
||||
private DataChecksum checksum;
|
||||
private final boolean verifyChecksum;
|
||||
|
||||
private static DirectBufferPool bufferPool = new DirectBufferPool();
|
||||
|
||||
|
@ -163,187 +84,92 @@ class BlockReaderLocal implements BlockReader {
|
|||
/** offset in block where reader wants to actually read */
|
||||
private long startOffset;
|
||||
private final String filename;
|
||||
|
||||
private final DatanodeID datanodeID;
|
||||
private final ExtendedBlock block;
|
||||
|
||||
/**
|
||||
* The only way this object can be instantiated.
|
||||
*/
|
||||
static BlockReaderLocal newBlockReader(UserGroupInformation ugi,
|
||||
Configuration conf, String file, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> token, DatanodeInfo node, int socketTimeout,
|
||||
long startOffset, long length, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
private static int getSlowReadBufferNumChunks(Configuration conf,
|
||||
int bytesPerChecksum) {
|
||||
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
||||
.getIpcPort());
|
||||
// check the cache first
|
||||
BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
|
||||
if (pathinfo == null) {
|
||||
pathinfo = getBlockPathInfo(ugi, blk, node, conf, socketTimeout, token,
|
||||
connectToDnViaHostname);
|
||||
}
|
||||
int bufSize =
|
||||
conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
||||
|
||||
// check to see if the file exists. It may so happen that the
|
||||
// HDFS file has been deleted and this block-lookup is occurring
|
||||
// on behalf of a new HDFS file. This time, the block file could
|
||||
// be residing in a different portion of the fs.data.dir directory.
|
||||
// In this case, we remove this entry from the cache. The next
|
||||
// call to this method will re-populate the cache.
|
||||
FileInputStream dataIn = null;
|
||||
FileInputStream checksumIn = null;
|
||||
BlockReaderLocal localBlockReader = null;
|
||||
boolean skipChecksumCheck = skipChecksumCheck(conf);
|
||||
try {
|
||||
// get a local file system
|
||||
File blkfile = new File(pathinfo.getBlockPath());
|
||||
dataIn = new FileInputStream(blkfile);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("New BlockReaderLocal for file " + blkfile + " of size "
|
||||
+ blkfile.length() + " startOffset " + startOffset + " length "
|
||||
+ length + " short circuit checksum " + !skipChecksumCheck);
|
||||
}
|
||||
|
||||
if (!skipChecksumCheck) {
|
||||
// get the metadata file
|
||||
File metafile = new File(pathinfo.getMetaPath());
|
||||
checksumIn = new FileInputStream(metafile);
|
||||
|
||||
// read and handle the common header here. For now just a version
|
||||
BlockMetadataHeader header = BlockMetadataHeader
|
||||
.readHeader(new DataInputStream(checksumIn));
|
||||
short version = header.getVersion();
|
||||
if (version != BlockMetadataHeader.VERSION) {
|
||||
LOG.warn("Wrong version (" + version + ") for metadata file for "
|
||||
+ blk + " ignoring ...");
|
||||
}
|
||||
DataChecksum checksum = header.getChecksum();
|
||||
long firstChunkOffset = startOffset
|
||||
- (startOffset % checksum.getBytesPerChecksum());
|
||||
localBlockReader = new BlockReaderLocal(conf, file, blk, token,
|
||||
startOffset, length, pathinfo, checksum, true, dataIn,
|
||||
firstChunkOffset, checksumIn);
|
||||
} else {
|
||||
localBlockReader = new BlockReaderLocal(conf, file, blk, token,
|
||||
startOffset, length, pathinfo, dataIn);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// remove from cache
|
||||
localDatanodeInfo.removeBlockLocalPathInfo(blk);
|
||||
DFSClient.LOG.warn("BlockReaderLocal: Removing " + blk
|
||||
+ " from cache because local file " + pathinfo.getBlockPath()
|
||||
+ " could not be opened.");
|
||||
throw e;
|
||||
} finally {
|
||||
if (localBlockReader == null) {
|
||||
if (dataIn != null) {
|
||||
dataIn.close();
|
||||
}
|
||||
if (checksumIn != null) {
|
||||
checksumIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
return localBlockReader;
|
||||
}
|
||||
|
||||
private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
|
||||
LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
|
||||
if (ldInfo == null) {
|
||||
ldInfo = new LocalDatanodeInfo();
|
||||
localDatanodeInfoMap.put(port, ldInfo);
|
||||
}
|
||||
return ldInfo;
|
||||
}
|
||||
|
||||
private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
|
||||
ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
|
||||
Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
|
||||
BlockLocalPathInfo pathinfo = null;
|
||||
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
|
||||
conf, timeout, connectToDnViaHostname);
|
||||
try {
|
||||
// make RPC to local datanode to find local pathnames of blocks
|
||||
pathinfo = proxy.getBlockLocalPathInfo(blk, token);
|
||||
if (pathinfo != null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Cached location of block " + blk + " as " + pathinfo);
|
||||
}
|
||||
localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
|
||||
throw e;
|
||||
}
|
||||
return pathinfo;
|
||||
}
|
||||
|
||||
private static boolean skipChecksumCheck(Configuration conf) {
|
||||
return conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
|
||||
}
|
||||
|
||||
private static int getSlowReadBufferNumChunks(Configuration conf, int bytesPerChecksum) {
|
||||
int bufferSizeBytes = conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
||||
|
||||
if (bufferSizeBytes < bytesPerChecksum) {
|
||||
throw new IllegalArgumentException("Configured BlockReaderLocal buffer size (" + bufferSizeBytes + ") " +
|
||||
"is not large enough to hold a single chunk (" + bytesPerChecksum + "). Please configure " +
|
||||
if (bufSize < bytesPerChecksum) {
|
||||
throw new IllegalArgumentException("Configured BlockReaderLocal buffer size (" +
|
||||
bufSize + ") is not large enough to hold a single chunk (" +
|
||||
bytesPerChecksum + "). Please configure " +
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY + " appropriately");
|
||||
}
|
||||
|
||||
// Round down to nearest chunk size
|
||||
return bufferSizeBytes / bytesPerChecksum;
|
||||
return bufSize / bytesPerChecksum;
|
||||
}
|
||||
|
||||
private BlockReaderLocal(Configuration conf, String hdfsfile,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
||||
long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
|
||||
throws IOException {
|
||||
this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
|
||||
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
||||
dataIn, startOffset, null);
|
||||
}
|
||||
|
||||
private BlockReaderLocal(Configuration conf, String hdfsfile,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
||||
long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
|
||||
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
||||
FileInputStream checksumIn) throws IOException {
|
||||
this.filename = hdfsfile;
|
||||
this.checksum = checksum;
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
this.startOffset = Math.max(startOffset, 0);
|
||||
|
||||
bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
||||
checksumSize = this.checksum.getChecksumSize();
|
||||
|
||||
public BlockReaderLocal(Configuration conf, String filename,
|
||||
ExtendedBlock block, long startOffset, long length,
|
||||
FileInputStream dataIn, FileInputStream checksumIn,
|
||||
DatanodeID datanodeID, boolean verifyChecksum) throws IOException {
|
||||
this.dataIn = dataIn;
|
||||
this.checksumIn = checksumIn;
|
||||
this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
|
||||
this.startOffset = Math.max(startOffset, 0);
|
||||
this.filename = filename;
|
||||
this.datanodeID = datanodeID;
|
||||
this.block = block;
|
||||
|
||||
int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
|
||||
slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
|
||||
checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
|
||||
// Initially the buffers have nothing to read.
|
||||
slowReadBuff.flip();
|
||||
checksumBuff.flip();
|
||||
// read and handle the common header here. For now just a version
|
||||
checksumIn.getChannel().position(0);
|
||||
BlockMetadataHeader header = BlockMetadataHeader
|
||||
.readHeader(new DataInputStream(
|
||||
new BufferedInputStream(checksumIn,
|
||||
BlockMetadataHeader.getHeaderSize())));
|
||||
short version = header.getVersion();
|
||||
if (version != BlockMetadataHeader.VERSION) {
|
||||
throw new IOException("Wrong version (" + version + ") of the " +
|
||||
"metadata file for " + filename + ".");
|
||||
}
|
||||
if (!verifyChecksum) {
|
||||
this.verifyChecksum = false;
|
||||
} else {
|
||||
this.verifyChecksum = !conf.getBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
|
||||
}
|
||||
long firstChunkOffset;
|
||||
if (this.verifyChecksum) {
|
||||
this.checksum = header.getChecksum();
|
||||
this.bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
||||
this.checksumSize = this.checksum.getChecksumSize();
|
||||
firstChunkOffset = startOffset
|
||||
- (startOffset % checksum.getBytesPerChecksum());
|
||||
this.offsetFromChunkBoundary = (int) (startOffset - firstChunkOffset);
|
||||
|
||||
int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
|
||||
slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
|
||||
checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
|
||||
// Initially the buffers have nothing to read.
|
||||
slowReadBuff.flip();
|
||||
checksumBuff.flip();
|
||||
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
|
||||
IOUtils.skipFully(checksumIn, checkSumOffset);
|
||||
} else {
|
||||
firstChunkOffset = startOffset;
|
||||
this.checksum = null;
|
||||
this.bytesPerChecksum = 0;
|
||||
this.checksumSize = 0;
|
||||
this.offsetFromChunkBoundary = 0;
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
// Skip both input streams to beginning of the chunk containing startOffset
|
||||
IOUtils.skipFully(dataIn, firstChunkOffset);
|
||||
if (checksumIn != null) {
|
||||
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
|
||||
IOUtils.skipFully(checksumIn, checkSumOffset);
|
||||
}
|
||||
// Reposition both input streams to the beginning of the chunk
|
||||
// containing startOffset
|
||||
this.dataIn.getChannel().position(firstChunkOffset);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
bufferPool.returnBuffer(slowReadBuff);
|
||||
bufferPool.returnBuffer(checksumBuff);
|
||||
if (slowReadBuff != null) bufferPool.returnBuffer(slowReadBuff);
|
||||
if (checksumBuff != null) bufferPool.returnBuffer(checksumBuff);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -663,10 +489,17 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
dataIn.close();
|
||||
if (checksumIn != null) {
|
||||
checksumIn.close();
|
||||
public synchronized void close(PeerCache peerCache,
|
||||
FileInputStreamCache fisCache) throws IOException {
|
||||
if (fisCache != null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("putting FileInputStream for " + filename +
|
||||
" back into FileInputStreamCache");
|
||||
}
|
||||
fisCache.put(datanodeID, block, new FileInputStream[] {dataIn, checksumIn});
|
||||
} else {
|
||||
LOG.debug("closing FileInputStream for " + filename);
|
||||
IOUtils.cleanup(LOG, dataIn, checksumIn);
|
||||
}
|
||||
if (slowReadBuff != null) {
|
||||
bufferPool.returnBuffer(slowReadBuff);
|
||||
|
@ -691,17 +524,8 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Socket takeSocket() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasSentStatusCode() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IOStreamPair getStreams() {
|
||||
return null;
|
||||
public int available() throws IOException {
|
||||
// We never do network I/O in BlockReaderLocal.
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,704 @@
|
|||
/**
|
||||
* 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.DataInputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.Socket;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.hdfs.util.DirectBufferPool;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
/**
|
||||
* BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on
|
||||
* the same machine as the datanode, then the client can read files directly
|
||||
* from the local file system rather than going through the datanode for better
|
||||
* performance. <br>
|
||||
*
|
||||
* This is the legacy implementation based on HDFS-2246, which requires
|
||||
* permissions on the datanode to be set so that clients can directly access the
|
||||
* blocks. The new implementation based on HDFS-347 should be preferred on UNIX
|
||||
* systems where the required native code has been implemented.<br>
|
||||
*
|
||||
* {@link BlockReaderLocalLegacy} works as follows:
|
||||
* <ul>
|
||||
* <li>The client performing short circuit reads must be configured at the
|
||||
* datanode.</li>
|
||||
* <li>The client gets the path to the file where block is stored using
|
||||
* {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
|
||||
* RPC call</li>
|
||||
* <li>Client uses kerberos authentication to connect to the datanode over RPC,
|
||||
* if security is enabled.</li>
|
||||
* </ul>
|
||||
*/
|
||||
class BlockReaderLocalLegacy implements BlockReader {
|
||||
private static final Log LOG = LogFactory.getLog(DFSClient.class);
|
||||
|
||||
//Stores the cache and proxy for a local datanode.
|
||||
private static class LocalDatanodeInfo {
|
||||
private ClientDatanodeProtocol proxy = null;
|
||||
private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
|
||||
|
||||
LocalDatanodeInfo() {
|
||||
final int cacheSize = 10000;
|
||||
final float hashTableLoadFactor = 0.75f;
|
||||
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
|
||||
cache = Collections
|
||||
.synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
|
||||
hashTableCapacity, hashTableLoadFactor, true) {
|
||||
private static final long serialVersionUID = 1;
|
||||
|
||||
@Override
|
||||
protected boolean removeEldestEntry(
|
||||
Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
|
||||
return size() > cacheSize;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private synchronized ClientDatanodeProtocol getDatanodeProxy(
|
||||
UserGroupInformation ugi, final DatanodeInfo node,
|
||||
final Configuration conf, final int socketTimeout,
|
||||
final boolean connectToDnViaHostname) throws IOException {
|
||||
if (proxy == null) {
|
||||
try {
|
||||
proxy = ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
|
||||
@Override
|
||||
public ClientDatanodeProtocol run() throws Exception {
|
||||
return DFSUtil.createClientDatanodeProtocolProxy(node, conf,
|
||||
socketTimeout, connectToDnViaHostname);
|
||||
}
|
||||
});
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("encountered exception ", e);
|
||||
}
|
||||
}
|
||||
return proxy;
|
||||
}
|
||||
|
||||
private synchronized void resetDatanodeProxy() {
|
||||
if (null != proxy) {
|
||||
RPC.stopProxy(proxy);
|
||||
proxy = null;
|
||||
}
|
||||
}
|
||||
|
||||
private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
|
||||
return cache.get(b);
|
||||
}
|
||||
|
||||
private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
|
||||
cache.put(b, info);
|
||||
}
|
||||
|
||||
private void removeBlockLocalPathInfo(ExtendedBlock b) {
|
||||
cache.remove(b);
|
||||
}
|
||||
}
|
||||
|
||||
// Multiple datanodes could be running on the local machine. Store proxies in
|
||||
// a map keyed by the ipc port of the datanode.
|
||||
private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
|
||||
|
||||
private final FileInputStream dataIn; // reader for the data file
|
||||
private final FileInputStream checksumIn; // reader for the checksum file
|
||||
|
||||
/**
|
||||
* Offset from the most recent chunk boundary at which the next read should
|
||||
* take place. Is only set to non-zero at construction time, and is
|
||||
* decremented (usually to 0) by subsequent reads. This avoids having to do a
|
||||
* checksum read at construction to position the read cursor correctly.
|
||||
*/
|
||||
private int offsetFromChunkBoundary;
|
||||
|
||||
private byte[] skipBuf = null;
|
||||
|
||||
/**
|
||||
* Used for checksummed reads that need to be staged before copying to their
|
||||
* output buffer because they are either a) smaller than the checksum chunk
|
||||
* size or b) issued by the slower read(byte[]...) path
|
||||
*/
|
||||
private ByteBuffer slowReadBuff = null;
|
||||
private ByteBuffer checksumBuff = null;
|
||||
private DataChecksum checksum;
|
||||
private final boolean verifyChecksum;
|
||||
|
||||
private static DirectBufferPool bufferPool = new DirectBufferPool();
|
||||
|
||||
private final int bytesPerChecksum;
|
||||
private final int checksumSize;
|
||||
|
||||
/** offset in block where reader wants to actually read */
|
||||
private long startOffset;
|
||||
private final String filename;
|
||||
|
||||
/**
|
||||
* The only way this object can be instantiated.
|
||||
*/
|
||||
static BlockReaderLocalLegacy newBlockReader(UserGroupInformation ugi,
|
||||
Configuration conf, String file, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> token, DatanodeInfo node, int socketTimeout,
|
||||
long startOffset, long length, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
||||
.getIpcPort());
|
||||
// check the cache first
|
||||
BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
|
||||
if (pathinfo == null) {
|
||||
pathinfo = getBlockPathInfo(ugi, blk, node, conf, socketTimeout, token,
|
||||
connectToDnViaHostname);
|
||||
}
|
||||
|
||||
// check to see if the file exists. It may so happen that the
|
||||
// HDFS file has been deleted and this block-lookup is occurring
|
||||
// on behalf of a new HDFS file. This time, the block file could
|
||||
// be residing in a different portion of the fs.data.dir directory.
|
||||
// In this case, we remove this entry from the cache. The next
|
||||
// call to this method will re-populate the cache.
|
||||
FileInputStream dataIn = null;
|
||||
FileInputStream checksumIn = null;
|
||||
BlockReaderLocalLegacy localBlockReader = null;
|
||||
boolean skipChecksumCheck = skipChecksumCheck(conf);
|
||||
try {
|
||||
// get a local file system
|
||||
File blkfile = new File(pathinfo.getBlockPath());
|
||||
dataIn = new FileInputStream(blkfile);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
|
||||
+ blkfile.length() + " startOffset " + startOffset + " length "
|
||||
+ length + " short circuit checksum " + !skipChecksumCheck);
|
||||
}
|
||||
|
||||
if (!skipChecksumCheck) {
|
||||
// get the metadata file
|
||||
File metafile = new File(pathinfo.getMetaPath());
|
||||
checksumIn = new FileInputStream(metafile);
|
||||
|
||||
// read and handle the common header here. For now just a version
|
||||
BlockMetadataHeader header = BlockMetadataHeader
|
||||
.readHeader(new DataInputStream(checksumIn));
|
||||
short version = header.getVersion();
|
||||
if (version != BlockMetadataHeader.VERSION) {
|
||||
LOG.warn("Wrong version (" + version + ") for metadata file for "
|
||||
+ blk + " ignoring ...");
|
||||
}
|
||||
DataChecksum checksum = header.getChecksum();
|
||||
long firstChunkOffset = startOffset
|
||||
- (startOffset % checksum.getBytesPerChecksum());
|
||||
localBlockReader = new BlockReaderLocalLegacy(conf, file, blk, token,
|
||||
startOffset, length, pathinfo, checksum, true, dataIn,
|
||||
firstChunkOffset, checksumIn);
|
||||
} else {
|
||||
localBlockReader = new BlockReaderLocalLegacy(conf, file, blk, token,
|
||||
startOffset, length, pathinfo, dataIn);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// remove from cache
|
||||
localDatanodeInfo.removeBlockLocalPathInfo(blk);
|
||||
DFSClient.LOG.warn("BlockReaderLocalLegacy: Removing " + blk
|
||||
+ " from cache because local file " + pathinfo.getBlockPath()
|
||||
+ " could not be opened.");
|
||||
throw e;
|
||||
} finally {
|
||||
if (localBlockReader == null) {
|
||||
if (dataIn != null) {
|
||||
dataIn.close();
|
||||
}
|
||||
if (checksumIn != null) {
|
||||
checksumIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
return localBlockReader;
|
||||
}
|
||||
|
||||
private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
|
||||
LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
|
||||
if (ldInfo == null) {
|
||||
ldInfo = new LocalDatanodeInfo();
|
||||
localDatanodeInfoMap.put(port, ldInfo);
|
||||
}
|
||||
return ldInfo;
|
||||
}
|
||||
|
||||
private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
|
||||
ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
|
||||
Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
|
||||
BlockLocalPathInfo pathinfo = null;
|
||||
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
|
||||
conf, timeout, connectToDnViaHostname);
|
||||
try {
|
||||
// make RPC to local datanode to find local pathnames of blocks
|
||||
pathinfo = proxy.getBlockLocalPathInfo(blk, token);
|
||||
if (pathinfo != null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Cached location of block " + blk + " as " + pathinfo);
|
||||
}
|
||||
localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
|
||||
throw e;
|
||||
}
|
||||
return pathinfo;
|
||||
}
|
||||
|
||||
private static boolean skipChecksumCheck(Configuration conf) {
|
||||
return conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
|
||||
}
|
||||
|
||||
private static int getSlowReadBufferNumChunks(Configuration conf, int bytesPerChecksum) {
|
||||
int bufferSizeBytes = conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
||||
|
||||
if (bufferSizeBytes < bytesPerChecksum) {
|
||||
throw new IllegalArgumentException("Configured BlockReaderLocalLegacy " +
|
||||
"buffer size (" + bufferSizeBytes + ") is not large enough to hold " +
|
||||
"a single chunk (" + bytesPerChecksum + "). Please configure " +
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY +
|
||||
" appropriately");
|
||||
}
|
||||
|
||||
// Round down to nearest chunk size
|
||||
return bufferSizeBytes / bytesPerChecksum;
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(Configuration conf, String hdfsfile,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
||||
long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
|
||||
throws IOException {
|
||||
this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
|
||||
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
||||
dataIn, startOffset, null);
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(Configuration conf, String hdfsfile,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
||||
long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
|
||||
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
||||
FileInputStream checksumIn) throws IOException {
|
||||
this.filename = hdfsfile;
|
||||
this.checksum = checksum;
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
this.startOffset = Math.max(startOffset, 0);
|
||||
|
||||
bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
||||
checksumSize = this.checksum.getChecksumSize();
|
||||
|
||||
this.dataIn = dataIn;
|
||||
this.checksumIn = checksumIn;
|
||||
this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
|
||||
|
||||
int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
|
||||
slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
|
||||
checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
|
||||
// Initially the buffers have nothing to read.
|
||||
slowReadBuff.flip();
|
||||
checksumBuff.flip();
|
||||
boolean success = false;
|
||||
try {
|
||||
// Skip both input streams to beginning of the chunk containing startOffset
|
||||
IOUtils.skipFully(dataIn, firstChunkOffset);
|
||||
if (checksumIn != null) {
|
||||
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
|
||||
IOUtils.skipFully(checksumIn, checkSumOffset);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
bufferPool.returnBuffer(slowReadBuff);
|
||||
bufferPool.returnBuffer(checksumBuff);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads bytes into a buffer until EOF or the buffer's limit is reached
|
||||
*/
|
||||
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
||||
throws IOException {
|
||||
int bytesRead = stream.getChannel().read(buf);
|
||||
if (bytesRead < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
while (buf.remaining() > 0) {
|
||||
int n = stream.getChannel().read(buf);
|
||||
if (n < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
bytesRead += n;
|
||||
}
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
|
||||
* another.
|
||||
*/
|
||||
private void writeSlice(ByteBuffer from, ByteBuffer to, int length) {
|
||||
int oldLimit = from.limit();
|
||||
from.limit(from.position() + length);
|
||||
try {
|
||||
to.put(from);
|
||||
} finally {
|
||||
from.limit(oldLimit);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(ByteBuffer buf) throws IOException {
|
||||
int nRead = 0;
|
||||
if (verifyChecksum) {
|
||||
// A 'direct' read actually has three phases. The first drains any
|
||||
// remaining bytes from the slow read buffer. After this the read is
|
||||
// guaranteed to be on a checksum chunk boundary. If there are still bytes
|
||||
// to read, the fast direct path is used for as many remaining bytes as
|
||||
// possible, up to a multiple of the checksum chunk size. Finally, any
|
||||
// 'odd' bytes remaining at the end of the read cause another slow read to
|
||||
// be issued, which involves an extra copy.
|
||||
|
||||
// Every 'slow' read tries to fill the slow read buffer in one go for
|
||||
// efficiency's sake. As described above, all non-checksum-chunk-aligned
|
||||
// reads will be served from the slower read path.
|
||||
|
||||
if (slowReadBuff.hasRemaining()) {
|
||||
// There are remaining bytes from a small read available. This usually
|
||||
// means this read is unaligned, which falls back to the slow path.
|
||||
int fromSlowReadBuff = Math.min(buf.remaining(), slowReadBuff.remaining());
|
||||
writeSlice(slowReadBuff, buf, fromSlowReadBuff);
|
||||
nRead += fromSlowReadBuff;
|
||||
}
|
||||
|
||||
if (buf.remaining() >= bytesPerChecksum && offsetFromChunkBoundary == 0) {
|
||||
// Since we have drained the 'small read' buffer, we are guaranteed to
|
||||
// be chunk-aligned
|
||||
int len = buf.remaining() - (buf.remaining() % bytesPerChecksum);
|
||||
|
||||
// There's only enough checksum buffer space available to checksum one
|
||||
// entire slow read buffer. This saves keeping the number of checksum
|
||||
// chunks around.
|
||||
len = Math.min(len, slowReadBuff.capacity());
|
||||
int oldlimit = buf.limit();
|
||||
buf.limit(buf.position() + len);
|
||||
int readResult = 0;
|
||||
try {
|
||||
readResult = doByteBufferRead(buf);
|
||||
} finally {
|
||||
buf.limit(oldlimit);
|
||||
}
|
||||
if (readResult == -1) {
|
||||
return nRead;
|
||||
} else {
|
||||
nRead += readResult;
|
||||
buf.position(buf.position() + readResult);
|
||||
}
|
||||
}
|
||||
|
||||
// offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
|
||||
// until chunk boundary
|
||||
if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) {
|
||||
int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary);
|
||||
int readResult = fillSlowReadBuffer(toRead);
|
||||
if (readResult == -1) {
|
||||
return nRead;
|
||||
} else {
|
||||
int fromSlowReadBuff = Math.min(readResult, buf.remaining());
|
||||
writeSlice(slowReadBuff, buf, fromSlowReadBuff);
|
||||
nRead += fromSlowReadBuff;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Non-checksummed reads are much easier; we can just fill the buffer directly.
|
||||
nRead = doByteBufferRead(buf);
|
||||
if (nRead > 0) {
|
||||
buf.position(buf.position() + nRead);
|
||||
}
|
||||
}
|
||||
return nRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tries to read as many bytes as possible into supplied buffer, checksumming
|
||||
* each chunk if needed.
|
||||
*
|
||||
* <b>Preconditions:</b>
|
||||
* <ul>
|
||||
* <li>
|
||||
* If checksumming is enabled, buf.remaining must be a multiple of
|
||||
* bytesPerChecksum. Note that this is not a requirement for clients of
|
||||
* read(ByteBuffer) - in the case of non-checksum-sized read requests,
|
||||
* read(ByteBuffer) will substitute a suitably sized buffer to pass to this
|
||||
* method.
|
||||
* </li>
|
||||
* </ul>
|
||||
* <b>Postconditions:</b>
|
||||
* <ul>
|
||||
* <li>buf.limit and buf.mark are unchanged.</li>
|
||||
* <li>buf.position += min(offsetFromChunkBoundary, totalBytesRead) - so the
|
||||
* requested bytes can be read straight from the buffer</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param buf
|
||||
* byte buffer to write bytes to. If checksums are not required, buf
|
||||
* can have any number of bytes remaining, otherwise there must be a
|
||||
* multiple of the checksum chunk size remaining.
|
||||
* @return <tt>max(min(totalBytesRead, len) - offsetFromChunkBoundary, 0)</tt>
|
||||
* that is, the the number of useful bytes (up to the amount
|
||||
* requested) readable from the buffer by the client.
|
||||
*/
|
||||
private synchronized int doByteBufferRead(ByteBuffer buf) throws IOException {
|
||||
if (verifyChecksum) {
|
||||
assert buf.remaining() % bytesPerChecksum == 0;
|
||||
}
|
||||
int dataRead = -1;
|
||||
|
||||
int oldpos = buf.position();
|
||||
// Read as much as we can into the buffer.
|
||||
dataRead = fillBuffer(dataIn, buf);
|
||||
|
||||
if (dataRead == -1) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
if (verifyChecksum) {
|
||||
ByteBuffer toChecksum = buf.duplicate();
|
||||
toChecksum.position(oldpos);
|
||||
toChecksum.limit(oldpos + dataRead);
|
||||
|
||||
checksumBuff.clear();
|
||||
// Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
|
||||
int numChunks =
|
||||
(toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
|
||||
checksumBuff.limit(checksumSize * numChunks);
|
||||
|
||||
fillBuffer(checksumIn, checksumBuff);
|
||||
checksumBuff.flip();
|
||||
|
||||
checksum.verifyChunkedSums(toChecksum, checksumBuff, filename,
|
||||
this.startOffset);
|
||||
}
|
||||
|
||||
if (dataRead >= 0) {
|
||||
buf.position(oldpos + Math.min(offsetFromChunkBoundary, dataRead));
|
||||
}
|
||||
|
||||
if (dataRead < offsetFromChunkBoundary) {
|
||||
// yikes, didn't even get enough bytes to honour offset. This can happen
|
||||
// even if we are verifying checksums if we are at EOF.
|
||||
offsetFromChunkBoundary -= dataRead;
|
||||
dataRead = 0;
|
||||
} else {
|
||||
dataRead -= offsetFromChunkBoundary;
|
||||
offsetFromChunkBoundary = 0;
|
||||
}
|
||||
|
||||
return dataRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures that up to len bytes are available and checksummed in the slow read
|
||||
* buffer. The number of bytes available to read is returned. If the buffer is
|
||||
* not already empty, the number of remaining bytes is returned and no actual
|
||||
* read happens.
|
||||
*
|
||||
* @param len
|
||||
* the maximum number of bytes to make available. After len bytes
|
||||
* are read, the underlying bytestream <b>must</b> be at a checksum
|
||||
* boundary, or EOF. That is, (len + currentPosition) %
|
||||
* bytesPerChecksum == 0.
|
||||
* @return the number of bytes available to read, or -1 if EOF.
|
||||
*/
|
||||
private synchronized int fillSlowReadBuffer(int len) throws IOException {
|
||||
int nRead = -1;
|
||||
if (slowReadBuff.hasRemaining()) {
|
||||
// Already got data, good to go.
|
||||
nRead = Math.min(len, slowReadBuff.remaining());
|
||||
} else {
|
||||
// Round a complete read of len bytes (plus any implicit offset) to the
|
||||
// next chunk boundary, since we try and read in multiples of a chunk
|
||||
int nextChunk = len + offsetFromChunkBoundary +
|
||||
(bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
|
||||
int limit = Math.min(nextChunk, slowReadBuff.capacity());
|
||||
assert limit % bytesPerChecksum == 0;
|
||||
|
||||
slowReadBuff.clear();
|
||||
slowReadBuff.limit(limit);
|
||||
|
||||
nRead = doByteBufferRead(slowReadBuff);
|
||||
|
||||
if (nRead > 0) {
|
||||
// So that next time we call slowReadBuff.hasRemaining(), we don't get a
|
||||
// false positive.
|
||||
slowReadBuff.limit(nRead + slowReadBuff.position());
|
||||
}
|
||||
}
|
||||
return nRead;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(byte[] buf, int off, int len) throws IOException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("read off " + off + " len " + len);
|
||||
}
|
||||
if (!verifyChecksum) {
|
||||
return dataIn.read(buf, off, len);
|
||||
}
|
||||
|
||||
int nRead = fillSlowReadBuffer(slowReadBuff.capacity());
|
||||
|
||||
if (nRead > 0) {
|
||||
// Possible that buffer is filled with a larger read than we need, since
|
||||
// we tried to read as much as possible at once
|
||||
nRead = Math.min(len, nRead);
|
||||
slowReadBuff.get(buf, off, nRead);
|
||||
}
|
||||
|
||||
return nRead;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long skip(long n) throws IOException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("skip " + n);
|
||||
}
|
||||
if (n <= 0) {
|
||||
return 0;
|
||||
}
|
||||
if (!verifyChecksum) {
|
||||
return dataIn.skip(n);
|
||||
}
|
||||
|
||||
// caller made sure newPosition is not beyond EOF.
|
||||
int remaining = slowReadBuff.remaining();
|
||||
int position = slowReadBuff.position();
|
||||
int newPosition = position + (int)n;
|
||||
|
||||
// if the new offset is already read into dataBuff, just reposition
|
||||
if (n <= remaining) {
|
||||
assert offsetFromChunkBoundary == 0;
|
||||
slowReadBuff.position(newPosition);
|
||||
return n;
|
||||
}
|
||||
|
||||
// for small gap, read through to keep the data/checksum in sync
|
||||
if (n - remaining <= bytesPerChecksum) {
|
||||
slowReadBuff.position(position + remaining);
|
||||
if (skipBuf == null) {
|
||||
skipBuf = new byte[bytesPerChecksum];
|
||||
}
|
||||
int ret = read(skipBuf, 0, (int)(n - remaining));
|
||||
return ret;
|
||||
}
|
||||
|
||||
// optimize for big gap: discard the current buffer, skip to
|
||||
// the beginning of the appropriate checksum chunk and then
|
||||
// read to the middle of that chunk to be in sync with checksums.
|
||||
|
||||
// We can't use this.offsetFromChunkBoundary because we need to know how
|
||||
// many bytes of the offset were really read. Calling read(..) with a
|
||||
// positive this.offsetFromChunkBoundary causes that many bytes to get
|
||||
// silently skipped.
|
||||
int myOffsetFromChunkBoundary = newPosition % bytesPerChecksum;
|
||||
long toskip = n - remaining - myOffsetFromChunkBoundary;
|
||||
|
||||
slowReadBuff.position(slowReadBuff.limit());
|
||||
checksumBuff.position(checksumBuff.limit());
|
||||
|
||||
IOUtils.skipFully(dataIn, toskip);
|
||||
long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
|
||||
IOUtils.skipFully(checksumIn, checkSumOffset);
|
||||
|
||||
// read into the middle of the chunk
|
||||
if (skipBuf == null) {
|
||||
skipBuf = new byte[bytesPerChecksum];
|
||||
}
|
||||
assert skipBuf.length == bytesPerChecksum;
|
||||
assert myOffsetFromChunkBoundary < bytesPerChecksum;
|
||||
|
||||
int ret = read(skipBuf, 0, myOffsetFromChunkBoundary);
|
||||
|
||||
if (ret == -1) { // EOS
|
||||
return toskip;
|
||||
} else {
|
||||
return (toskip + ret);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close(PeerCache peerCache,
|
||||
FileInputStreamCache fisCache) throws IOException {
|
||||
IOUtils.cleanup(LOG, dataIn, checksumIn);
|
||||
if (slowReadBuff != null) {
|
||||
bufferPool.returnBuffer(slowReadBuff);
|
||||
slowReadBuff = null;
|
||||
}
|
||||
if (checksumBuff != null) {
|
||||
bufferPool.returnBuffer(checksumBuff);
|
||||
checksumBuff = null;
|
||||
}
|
||||
startOffset = -1;
|
||||
checksum = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readAll(byte[] buf, int offset, int len) throws IOException {
|
||||
return BlockReaderUtil.readAll(this, buf, offset, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(byte[] buf, int off, int len) throws IOException {
|
||||
BlockReaderUtil.readFully(this, buf, off, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
// We never do network I/O in BlockReaderLocalLegacy.
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
|
@ -129,7 +129,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
|
|||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
|
@ -193,12 +192,13 @@ public class DFSClient implements java.io.Closeable {
|
|||
final FileSystem.Statistics stats;
|
||||
final int hdfsTimeout; // timeout value for a DFS operation.
|
||||
private final String authority;
|
||||
final SocketCache socketCache;
|
||||
final PeerCache peerCache;
|
||||
final Conf dfsClientConf;
|
||||
private Random r = new Random();
|
||||
private SocketAddress[] localInterfaceAddrs;
|
||||
private DataEncryptionKey encryptionKey;
|
||||
|
||||
private boolean shouldUseLegacyBlockReaderLocal;
|
||||
|
||||
/**
|
||||
* DFSClient configuration
|
||||
*/
|
||||
|
@ -225,11 +225,16 @@ public class DFSClient implements java.io.Closeable {
|
|||
final short defaultReplication;
|
||||
final String taskId;
|
||||
final FsPermission uMask;
|
||||
final boolean useLegacyBlockReader;
|
||||
final boolean useLegacyBlockReaderLocal;
|
||||
final boolean connectToDnViaHostname;
|
||||
final boolean getHdfsBlocksMetadataEnabled;
|
||||
final int getFileBlockStorageLocationsNumThreads;
|
||||
final int getFileBlockStorageLocationsTimeout;
|
||||
final String domainSocketPath;
|
||||
final boolean skipShortCircuitChecksums;
|
||||
final int shortCircuitBufferSize;
|
||||
final boolean shortCircuitLocalReads;
|
||||
final boolean domainSocketDataTraffic;
|
||||
|
||||
Conf(Configuration conf) {
|
||||
maxFailoverAttempts = conf.getInt(
|
||||
|
@ -280,9 +285,9 @@ public class DFSClient implements java.io.Closeable {
|
|||
.getInt(DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
|
||||
DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
|
||||
uMask = FsPermission.getUMask(conf);
|
||||
useLegacyBlockReader = conf.getBoolean(
|
||||
DFS_CLIENT_USE_LEGACY_BLOCKREADER,
|
||||
DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT);
|
||||
useLegacyBlockReaderLocal = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
|
||||
DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
|
||||
connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
|
||||
DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
|
||||
getHdfsBlocksMetadataEnabled = conf.getBoolean(
|
||||
|
@ -294,6 +299,20 @@ public class DFSClient implements java.io.Closeable {
|
|||
getFileBlockStorageLocationsTimeout = conf.getInt(
|
||||
DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT,
|
||||
DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT);
|
||||
domainSocketPath = conf.getTrimmed(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
|
||||
skipShortCircuitChecksums = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
|
||||
shortCircuitBufferSize = conf.getInt(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
||||
shortCircuitLocalReads = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
|
||||
domainSocketDataTraffic = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
|
||||
DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
|
||||
}
|
||||
|
||||
private DataChecksum.Type getChecksumType(Configuration conf) {
|
||||
|
@ -351,7 +370,7 @@ public class DFSClient implements java.io.Closeable {
|
|||
private final Map<String, DFSOutputStream> filesBeingWritten
|
||||
= new HashMap<String, DFSOutputStream>();
|
||||
|
||||
private boolean shortCircuitLocalReads;
|
||||
private final DomainSocketFactory domainSocketFactory;
|
||||
|
||||
/**
|
||||
* Same as this(NameNode.getAddress(conf), conf);
|
||||
|
@ -395,6 +414,11 @@ public class DFSClient implements java.io.Closeable {
|
|||
throws IOException {
|
||||
// Copy only the required DFSClient configuration
|
||||
this.dfsClientConf = new Conf(conf);
|
||||
this.shouldUseLegacyBlockReaderLocal =
|
||||
this.dfsClientConf.useLegacyBlockReaderLocal;
|
||||
if (this.dfsClientConf.useLegacyBlockReaderLocal) {
|
||||
LOG.debug("Using legacy short-circuit local reads.");
|
||||
}
|
||||
this.conf = conf;
|
||||
this.stats = stats;
|
||||
this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
|
||||
|
@ -424,12 +448,8 @@ public class DFSClient implements java.io.Closeable {
|
|||
}
|
||||
|
||||
// read directly from the block file if configured.
|
||||
this.shortCircuitLocalReads = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Short circuit read is " + shortCircuitLocalReads);
|
||||
}
|
||||
this.domainSocketFactory = new DomainSocketFactory(dfsClientConf);
|
||||
|
||||
String localInterfaces[] =
|
||||
conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
|
||||
localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
|
||||
|
@ -439,7 +459,7 @@ public class DFSClient implements java.io.Closeable {
|
|||
Joiner.on(',').join(localInterfaceAddrs) + "]");
|
||||
}
|
||||
|
||||
this.socketCache = SocketCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
|
||||
this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -794,29 +814,11 @@ public class DFSClient implements java.io.Closeable {
|
|||
AccessControlException.class);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get {@link BlockReader} for short circuited local reads.
|
||||
*/
|
||||
static BlockReader getLocalBlockReader(UserGroupInformation ugi,
|
||||
Configuration conf, String src, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
|
||||
int socketTimeout, long offsetIntoBlock, boolean connectToDnViaHostname)
|
||||
throws InvalidToken, IOException {
|
||||
try {
|
||||
return BlockReaderLocal.newBlockReader(ugi, conf, src, blk, accessToken,
|
||||
chosenNode, socketTimeout, offsetIntoBlock, blk.getNumBytes()
|
||||
- offsetIntoBlock, connectToDnViaHostname);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException(InvalidToken.class,
|
||||
AccessControlException.class);
|
||||
}
|
||||
}
|
||||
|
||||
private static Map<String, Boolean> localAddrMap = Collections
|
||||
.synchronizedMap(new HashMap<String, Boolean>());
|
||||
|
||||
private static boolean isLocalAddress(InetSocketAddress targetAddr) {
|
||||
static boolean isLocalAddress(InetSocketAddress targetAddr) {
|
||||
InetAddress addr = targetAddr.getAddress();
|
||||
Boolean cached = localAddrMap.get(addr.getHostAddress());
|
||||
if (cached != null) {
|
||||
|
@ -2218,10 +2220,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
super(in);
|
||||
}
|
||||
}
|
||||
|
||||
boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr) {
|
||||
return shortCircuitLocalReads && isLocalAddress(targetAddr);
|
||||
}
|
||||
|
||||
void reportChecksumFailure(String file, ExtendedBlock blk, DatanodeInfo dn) {
|
||||
DatanodeInfo [] dnArr = { dn };
|
||||
|
@ -2245,13 +2243,15 @@ public class DFSClient implements java.io.Closeable {
|
|||
+ ", ugi=" + ugi + "]";
|
||||
}
|
||||
|
||||
void disableShortCircuit() {
|
||||
LOG.info("Short circuit is disabled");
|
||||
shortCircuitLocalReads = false;
|
||||
public DomainSocketFactory getDomainSocketFactory() {
|
||||
return domainSocketFactory;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean getShortCircuitLocalReads() {
|
||||
return shortCircuitLocalReads;
|
||||
|
||||
public void disableLegacyBlockReaderLocal() {
|
||||
shouldUseLegacyBlockReaderLocal = false;
|
||||
}
|
||||
|
||||
public boolean useLegacyBlockReaderLocal() {
|
||||
return shouldUseLegacyBlockReaderLocal;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -265,6 +265,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final int DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3;
|
||||
public static final String DFS_CLIENT_USE_LEGACY_BLOCKREADER = "dfs.client.use.legacy.blockreader";
|
||||
public static final boolean DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT = false;
|
||||
public static final String DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL = "dfs.client.use.legacy.blockreader.local";
|
||||
public static final boolean DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT = false;
|
||||
public static final String DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth";
|
||||
public static final long DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
|
||||
public static final String DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
|
||||
|
@ -347,7 +349,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY = "dfs.client.read.shortcircuit.skip.checksum";
|
||||
public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false;
|
||||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY = "dfs.client.read.shortcircuit.buffer.size";
|
||||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY = "dfs.client.read.shortcircuit.streams.cache.size";
|
||||
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 100;
|
||||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY = "dfs.client.read.shortcircuit.streams.cache.expiry.ms";
|
||||
public static final long DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5000;
|
||||
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT = 1024 * 1024;
|
||||
public static final String DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC = "dfs.client.domain.socket.data.traffic";
|
||||
public static final boolean DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT = false;
|
||||
|
||||
// property for fsimage compression
|
||||
public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";
|
||||
|
@ -404,6 +412,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final int DFS_NAMENODE_MAX_OP_SIZE_DEFAULT = 50 * 1024 * 1024;
|
||||
|
||||
public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
|
||||
public static final String DFS_DOMAIN_SOCKET_PATH_KEY = "dfs.domain.socket.path";
|
||||
public static final String DFS_DOMAIN_SOCKET_PATH_DEFAULT = "";
|
||||
|
||||
// HA related configuration
|
||||
public static final String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
|
@ -32,18 +33,20 @@ import java.util.Map.Entry;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.ByteBufferReadable;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.hdfs.SocketCache.SocketAndStreams;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||
|
@ -51,20 +54,23 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
|||
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 com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/****************************************************************
|
||||
* DFSInputStream provides bytes from a named file. It handles
|
||||
* negotiation of the namenode and various datanodes as necessary.
|
||||
****************************************************************/
|
||||
@InterfaceAudience.Private
|
||||
public class DFSInputStream extends FSInputStream implements ByteBufferReadable {
|
||||
private final SocketCache socketCache;
|
||||
|
||||
@VisibleForTesting
|
||||
static boolean tcpReadsDisabledForTesting = false;
|
||||
private final PeerCache peerCache;
|
||||
private final DFSClient dfsClient;
|
||||
private boolean closed = false;
|
||||
|
||||
private final String src;
|
||||
private final long prefetchSize;
|
||||
private BlockReader blockReader = null;
|
||||
|
@ -76,6 +82,8 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
private long pos = 0;
|
||||
private long blockEnd = -1;
|
||||
|
||||
private final FileInputStreamCache fileInputStreamCache;
|
||||
|
||||
/**
|
||||
* This variable tracks the number of failures since the start of the
|
||||
* most recent user-facing operation. That is to say, it should be reset
|
||||
|
@ -110,7 +118,14 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
this.verifyChecksum = verifyChecksum;
|
||||
this.buffersize = buffersize;
|
||||
this.src = src;
|
||||
this.socketCache = dfsClient.socketCache;
|
||||
this.peerCache = dfsClient.peerCache;
|
||||
this.fileInputStreamCache = new FileInputStreamCache(
|
||||
dfsClient.conf.getInt(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT),
|
||||
dfsClient.conf.getLong(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT));
|
||||
prefetchSize = dfsClient.getConf().prefetchSize;
|
||||
timeWindow = dfsClient.getConf().timeWindow;
|
||||
nCachedConnRetry = dfsClient.getConf().nCachedConnRetry;
|
||||
|
@ -243,7 +258,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
|
||||
}
|
||||
|
||||
private synchronized boolean blockUnderConstruction() {
|
||||
// Short circuit local reads are forbidden for files that are
|
||||
// under construction. See HDFS-2757.
|
||||
synchronized boolean shortCircuitForbidden() {
|
||||
return locatedBlocks.isUnderConstruction();
|
||||
}
|
||||
|
||||
|
@ -424,7 +441,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
|
||||
// Will be getting a new BlockReader.
|
||||
if (blockReader != null) {
|
||||
closeBlockReader(blockReader);
|
||||
blockReader.close(peerCache, fileInputStreamCache);
|
||||
blockReader = null;
|
||||
}
|
||||
|
||||
|
@ -462,7 +479,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
return chosenNode;
|
||||
} catch (AccessControlException ex) {
|
||||
DFSClient.LOG.warn("Short circuit access failed " + ex);
|
||||
dfsClient.disableShortCircuit();
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
continue;
|
||||
} catch (IOException ex) {
|
||||
if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
|
@ -510,10 +527,11 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
dfsClient.checkOpen();
|
||||
|
||||
if (blockReader != null) {
|
||||
closeBlockReader(blockReader);
|
||||
blockReader.close(peerCache, fileInputStreamCache);
|
||||
blockReader = null;
|
||||
}
|
||||
super.close();
|
||||
fileInputStreamCache.close();
|
||||
closed = true;
|
||||
}
|
||||
|
||||
|
@ -811,7 +829,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
|
||||
} catch (AccessControlException ex) {
|
||||
DFSClient.LOG.warn("Short circuit access failed " + ex);
|
||||
dfsClient.disableShortCircuit();
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
continue;
|
||||
} catch (IOException e) {
|
||||
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
|
@ -837,7 +855,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
}
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
closeBlockReader(reader);
|
||||
reader.close(peerCache, fileInputStreamCache);
|
||||
}
|
||||
}
|
||||
// Put chosen node into dead list, continue
|
||||
|
@ -845,22 +863,34 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the given BlockReader and cache its socket.
|
||||
*/
|
||||
private void closeBlockReader(BlockReader reader) throws IOException {
|
||||
if (reader.hasSentStatusCode()) {
|
||||
IOStreamPair ioStreams = reader.getStreams();
|
||||
Socket oldSock = reader.takeSocket();
|
||||
socketCache.put(oldSock, ioStreams);
|
||||
private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
|
||||
Peer peer = null;
|
||||
boolean success = false;
|
||||
Socket sock = null;
|
||||
try {
|
||||
sock = dfsClient.socketFactory.createSocket();
|
||||
NetUtils.connect(sock, addr,
|
||||
dfsClient.getRandomLocalInterfaceAddr(),
|
||||
dfsClient.getConf().socketTimeout);
|
||||
peer = TcpPeerServer.peerFromSocketAndKey(sock,
|
||||
dfsClient.getDataEncryptionKey());
|
||||
success = true;
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
IOUtils.closeQuietly(sock);
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve a BlockReader suitable for reading.
|
||||
* This method will reuse the cached connection to the DN if appropriate.
|
||||
* Otherwise, it will create a new connection.
|
||||
* Throwing an IOException from this method is basically equivalent to
|
||||
* declaring the DataNode bad, so we try to connect a lot of different ways
|
||||
* before doing that.
|
||||
*
|
||||
* @param dnAddr Address of the datanode
|
||||
* @param chosenNode Chosen datanode information
|
||||
|
@ -885,82 +915,113 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
boolean verifyChecksum,
|
||||
String clientName)
|
||||
throws IOException {
|
||||
|
||||
// Can't local read a block under construction, see HDFS-2757
|
||||
if (dfsClient.shouldTryShortCircuitRead(dnAddr) &&
|
||||
!blockUnderConstruction()) {
|
||||
return DFSClient.getLocalBlockReader(dfsClient.ugi, dfsClient.conf,
|
||||
src, block, blockToken, chosenNode, dfsClient.hdfsTimeout,
|
||||
startOffset, dfsClient.connectToDnViaHostname());
|
||||
// Firstly, we check to see if we have cached any file descriptors for
|
||||
// local blocks. If so, we can just re-use those file descriptors.
|
||||
FileInputStream fis[] = fileInputStreamCache.get(chosenNode, block);
|
||||
if (fis != null) {
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
|
||||
"the FileInputStreamCache.");
|
||||
}
|
||||
return new BlockReaderLocal(dfsClient.conf, file,
|
||||
block, startOffset, len, fis[0], fis[1], chosenNode, verifyChecksum);
|
||||
}
|
||||
|
||||
IOException err = null;
|
||||
boolean fromCache = true;
|
||||
|
||||
// Allow retry since there is no way of knowing whether the cached socket
|
||||
// is good until we actually use it.
|
||||
for (int retries = 0; retries <= nCachedConnRetry && fromCache; ++retries) {
|
||||
SocketAndStreams sockAndStreams = null;
|
||||
// Don't use the cache on the last attempt - it's possible that there
|
||||
// are arbitrarily many unusable sockets in the cache, but we don't
|
||||
// want to fail the read.
|
||||
if (retries < nCachedConnRetry) {
|
||||
sockAndStreams = socketCache.get(dnAddr);
|
||||
}
|
||||
Socket sock;
|
||||
if (sockAndStreams == null) {
|
||||
fromCache = false;
|
||||
|
||||
sock = dfsClient.socketFactory.createSocket();
|
||||
|
||||
// TCP_NODELAY is crucial here because of bad interactions between
|
||||
// Nagle's Algorithm and Delayed ACKs. With connection keepalive
|
||||
// between the client and DN, the conversation looks like:
|
||||
// 1. Client -> DN: Read block X
|
||||
// 2. DN -> Client: data for block X
|
||||
// 3. Client -> DN: Status OK (successful read)
|
||||
// 4. Client -> DN: Read block Y
|
||||
// The fact that step #3 and #4 are both in the client->DN direction
|
||||
// triggers Nagling. If the DN is using delayed ACKs, this results
|
||||
// in a delay of 40ms or more.
|
||||
//
|
||||
// TCP_NODELAY disables nagling and thus avoids this performance
|
||||
// disaster.
|
||||
sock.setTcpNoDelay(true);
|
||||
|
||||
NetUtils.connect(sock, dnAddr,
|
||||
dfsClient.getRandomLocalInterfaceAddr(),
|
||||
dfsClient.getConf().socketTimeout);
|
||||
sock.setSoTimeout(dfsClient.getConf().socketTimeout);
|
||||
} else {
|
||||
sock = sockAndStreams.sock;
|
||||
}
|
||||
|
||||
// If the legacy local block reader is enabled and we are reading a local
|
||||
// block, try to create a BlockReaderLocalLegacy. The legacy local block
|
||||
// reader implements local reads in the style first introduced by HDFS-2246.
|
||||
if ((dfsClient.useLegacyBlockReaderLocal()) &&
|
||||
DFSClient.isLocalAddress(dnAddr) &&
|
||||
(!shortCircuitForbidden())) {
|
||||
try {
|
||||
// The OP_READ_BLOCK request is sent as we make the BlockReader
|
||||
BlockReader reader =
|
||||
BlockReaderFactory.newBlockReader(dfsClient.getConf(),
|
||||
sock, file, block,
|
||||
blockToken,
|
||||
startOffset, len,
|
||||
bufferSize, verifyChecksum,
|
||||
clientName,
|
||||
dfsClient.getDataEncryptionKey(),
|
||||
sockAndStreams == null ? null : sockAndStreams.ioStreams);
|
||||
return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient.ugi,
|
||||
dfsClient.conf, clientName, block, blockToken, chosenNode,
|
||||
dfsClient.hdfsTimeout, startOffset,dfsClient.connectToDnViaHostname());
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.warn("error creating legacy BlockReaderLocal. " +
|
||||
"Disabling legacy local reads.", e);
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
}
|
||||
}
|
||||
|
||||
// Look for cached domain peers.
|
||||
int cacheTries = 0;
|
||||
DomainSocketFactory dsFactory = dfsClient.getDomainSocketFactory();
|
||||
BlockReader reader = null;
|
||||
for (; cacheTries < nCachedConnRetry; ++cacheTries) {
|
||||
Peer peer = peerCache.get(chosenNode, true);
|
||||
if (peer == null) break;
|
||||
try {
|
||||
boolean allowShortCircuitLocalReads = dfsClient.getConf().
|
||||
shortCircuitLocalReads && (!shortCircuitForbidden());
|
||||
reader = BlockReaderFactory.newBlockReader(
|
||||
dfsClient.conf, file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, allowShortCircuitLocalReads);
|
||||
return reader;
|
||||
} catch (IOException ex) {
|
||||
// Our socket is no good.
|
||||
DFSClient.LOG.debug("Error making BlockReader. Closing stale " + sock, ex);
|
||||
if (sockAndStreams != null) {
|
||||
sockAndStreams.close();
|
||||
} else {
|
||||
sock.close();
|
||||
DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
|
||||
"Closing stale " + peer, ex);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
err = ex;
|
||||
}
|
||||
}
|
||||
|
||||
throw err;
|
||||
// Try to create a DomainPeer.
|
||||
DomainSocket domSock = dsFactory.create(dnAddr, this);
|
||||
if (domSock != null) {
|
||||
Peer peer = new DomainPeer(domSock);
|
||||
try {
|
||||
boolean allowShortCircuitLocalReads = dfsClient.getConf().
|
||||
shortCircuitLocalReads && (!shortCircuitForbidden());
|
||||
reader = BlockReaderFactory.newBlockReader(
|
||||
dfsClient.conf, file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, allowShortCircuitLocalReads);
|
||||
return reader;
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.warn("failed to connect to " + domSock, e);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
// If the Peer that we got the error from was a DomainPeer,
|
||||
// mark the socket path as bad, so that newDataSocket will not try
|
||||
// to re-open this socket for a while.
|
||||
dsFactory.disableDomainSocketPath(domSock.getPath());
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Look for cached peers.
|
||||
for (; cacheTries < nCachedConnRetry; ++cacheTries) {
|
||||
Peer peer = peerCache.get(chosenNode, false);
|
||||
if (peer == null) break;
|
||||
try {
|
||||
reader = BlockReaderFactory.newBlockReader(
|
||||
dfsClient.conf, file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, false);
|
||||
return reader;
|
||||
} catch (IOException ex) {
|
||||
DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
|
||||
peer, ex);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (tcpReadsDisabledForTesting) {
|
||||
throw new IOException("TCP reads are disabled.");
|
||||
}
|
||||
// Try to create a new remote peer.
|
||||
Peer peer = newTcpPeer(dnAddr);
|
||||
return BlockReaderFactory.newBlockReader(
|
||||
dfsClient.conf, file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, false);
|
||||
}
|
||||
|
||||
|
||||
|
@ -1094,7 +1155,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
// the TCP buffer, then just eat up the intervening data.
|
||||
//
|
||||
int diff = (int)(targetPos - pos);
|
||||
if (diff <= DFSClient.TCP_WINDOW_SIZE) {
|
||||
if (diff <= blockReader.available()) {
|
||||
try {
|
||||
pos += blockReader.skip(diff);
|
||||
if (pos == targetPos) {
|
||||
|
|
|
@ -0,0 +1,142 @@
|
|||
/**
|
||||
* 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.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
|
||||
import com.google.common.cache.Cache;
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
|
||||
class DomainSocketFactory {
|
||||
public static final Log LOG = LogFactory.getLog(DomainSocketFactory.class);
|
||||
private final Conf conf;
|
||||
|
||||
enum PathStatus {
|
||||
UNUSABLE,
|
||||
SHORT_CIRCUIT_DISABLED,
|
||||
}
|
||||
|
||||
/**
|
||||
* Information about domain socket paths.
|
||||
*/
|
||||
Cache<String, PathStatus> pathInfo =
|
||||
CacheBuilder.newBuilder()
|
||||
.expireAfterWrite(10, TimeUnit.MINUTES)
|
||||
.build();
|
||||
|
||||
public DomainSocketFactory(Conf conf) {
|
||||
this.conf = conf;
|
||||
|
||||
String feature = null;
|
||||
if (conf.shortCircuitLocalReads && (!conf.useLegacyBlockReaderLocal)) {
|
||||
feature = "The short-circuit local reads feature";
|
||||
} else if (conf.domainSocketDataTraffic) {
|
||||
feature = "UNIX domain socket data traffic";
|
||||
}
|
||||
if (feature != null) {
|
||||
if (conf.domainSocketPath.isEmpty()) {
|
||||
LOG.warn(feature + " is disabled because you have not set " +
|
||||
DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY);
|
||||
} else if (DomainSocket.getLoadingFailureReason() != null) {
|
||||
LOG.warn(feature + " is disabled because " +
|
||||
DomainSocket.getLoadingFailureReason());
|
||||
} else {
|
||||
LOG.debug(feature + "is enabled.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a DomainSocket.
|
||||
*
|
||||
* @param addr The address of the DataNode
|
||||
* @param stream The DFSInputStream the socket will be created for.
|
||||
*
|
||||
* @return null if the socket could not be created; the
|
||||
* socket otherwise. If there was an error while
|
||||
* creating the socket, we will add the socket path
|
||||
* to our list of failed domain socket paths.
|
||||
*/
|
||||
DomainSocket create(InetSocketAddress addr, DFSInputStream stream) {
|
||||
// If there is no domain socket path configured, we can't use domain
|
||||
// sockets.
|
||||
if (conf.domainSocketPath.isEmpty()) return null;
|
||||
// If we can't do anything with the domain socket, don't create it.
|
||||
if ((conf.domainSocketDataTraffic == false) &&
|
||||
((!conf.shortCircuitLocalReads) || conf.useLegacyBlockReaderLocal)) {
|
||||
return null;
|
||||
}
|
||||
// UNIX domain sockets can only be used to talk to local peers
|
||||
if (!DFSClient.isLocalAddress(addr)) return null;
|
||||
// If the DomainSocket code is not loaded, we can't create
|
||||
// DomainSocket objects.
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return null;
|
||||
String escapedPath = DomainSocket.
|
||||
getEffectivePath(conf.domainSocketPath, addr.getPort());
|
||||
PathStatus info = pathInfo.getIfPresent(escapedPath);
|
||||
if (info == PathStatus.UNUSABLE) {
|
||||
// We tried to connect to this domain socket before, and it was totally
|
||||
// unusable.
|
||||
return null;
|
||||
}
|
||||
if ((!conf.domainSocketDataTraffic) &&
|
||||
((info == PathStatus.SHORT_CIRCUIT_DISABLED) ||
|
||||
stream.shortCircuitForbidden())) {
|
||||
// If we don't want to pass data over domain sockets, and we don't want
|
||||
// to pass file descriptors over them either, we have no use for domain
|
||||
// sockets.
|
||||
return null;
|
||||
}
|
||||
boolean success = false;
|
||||
DomainSocket sock = null;
|
||||
try {
|
||||
sock = DomainSocket.connect(escapedPath);
|
||||
sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, conf.socketTimeout);
|
||||
success = true;
|
||||
} catch (IOException e) {
|
||||
LOG.warn("error creating DomainSocket", e);
|
||||
// fall through
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (sock != null) {
|
||||
IOUtils.closeQuietly(sock);
|
||||
}
|
||||
pathInfo.put(escapedPath, PathStatus.UNUSABLE);
|
||||
sock = null;
|
||||
}
|
||||
}
|
||||
return sock;
|
||||
}
|
||||
|
||||
public void disableShortCircuitForPath(String path) {
|
||||
pathInfo.put(path, PathStatus.SHORT_CIRCUIT_DISABLED);
|
||||
}
|
||||
|
||||
public void disableDomainSocketPath(String path) {
|
||||
pathInfo.put(path, PathStatus.UNUSABLE);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,264 @@
|
|||
/**
|
||||
* 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.FileInputStream;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.collect.LinkedListMultimap;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* FileInputStream cache is used to cache FileInputStream objects that we
|
||||
* have received from the DataNode.
|
||||
*/
|
||||
class FileInputStreamCache {
|
||||
private final static Log LOG = LogFactory.getLog(FileInputStreamCache.class);
|
||||
|
||||
/**
|
||||
* The executor service that runs the cacheCleaner. There is only one of
|
||||
* these per VM.
|
||||
*/
|
||||
private final static ScheduledThreadPoolExecutor executor
|
||||
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("FileInputStreamCache Cleaner").
|
||||
build());
|
||||
|
||||
/**
|
||||
* The CacheCleaner for this FileInputStreamCache. We don't create this
|
||||
* and schedule it until it becomes necessary.
|
||||
*/
|
||||
private CacheCleaner cacheCleaner;
|
||||
|
||||
/**
|
||||
* Maximum number of entries to allow in the cache.
|
||||
*/
|
||||
private final int maxCacheSize;
|
||||
|
||||
/**
|
||||
* The minimum time in milliseconds to preserve an element in the cache.
|
||||
*/
|
||||
private final long expiryTimeMs;
|
||||
|
||||
/**
|
||||
* True if the FileInputStreamCache is closed.
|
||||
*/
|
||||
private boolean closed = false;
|
||||
|
||||
/**
|
||||
* Cache entries.
|
||||
*/
|
||||
private final LinkedListMultimap<Key, Value> map = LinkedListMultimap.create();
|
||||
|
||||
/**
|
||||
* Expiry thread which makes sure that the file descriptors get closed
|
||||
* after a while.
|
||||
*/
|
||||
class CacheCleaner implements Runnable {
|
||||
@Override
|
||||
public void run() {
|
||||
synchronized(FileInputStreamCache.this) {
|
||||
if (closed) return;
|
||||
long curTime = Time.monotonicNow();
|
||||
for (Iterator<Entry<Key, Value>> iter = map.entries().iterator();
|
||||
iter.hasNext();
|
||||
iter = map.entries().iterator()) {
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
if (entry.getValue().getTime() + expiryTimeMs >= curTime) {
|
||||
break;
|
||||
}
|
||||
entry.getValue().close();
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The key identifying a FileInputStream array.
|
||||
*/
|
||||
static class Key {
|
||||
private final DatanodeID datanodeID;
|
||||
private final ExtendedBlock block;
|
||||
|
||||
public Key(DatanodeID datanodeID, ExtendedBlock block) {
|
||||
this.datanodeID = datanodeID;
|
||||
this.block = block;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (!(other instanceof FileInputStreamCache.Key)) {
|
||||
return false;
|
||||
}
|
||||
FileInputStreamCache.Key otherKey = (FileInputStreamCache.Key)other;
|
||||
return (block.equals(otherKey.block) &
|
||||
(block.getGenerationStamp() == otherKey.block.getGenerationStamp()) &
|
||||
datanodeID.equals(otherKey.datanodeID));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return block.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The value containing a FileInputStream array and the time it was added to
|
||||
* the cache.
|
||||
*/
|
||||
static class Value {
|
||||
private final FileInputStream fis[];
|
||||
private final long time;
|
||||
|
||||
public Value (FileInputStream fis[]) {
|
||||
this.fis = fis;
|
||||
this.time = Time.monotonicNow();
|
||||
}
|
||||
|
||||
public FileInputStream[] getFileInputStreams() {
|
||||
return fis;
|
||||
}
|
||||
|
||||
public long getTime() {
|
||||
return time;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new FileInputStream
|
||||
*
|
||||
* @param maxCacheSize The maximum number of elements to allow in
|
||||
* the cache.
|
||||
* @param expiryTimeMs The minimum time in milliseconds to preserve
|
||||
* elements in the cache.
|
||||
*/
|
||||
public FileInputStreamCache(int maxCacheSize, long expiryTimeMs) {
|
||||
this.maxCacheSize = maxCacheSize;
|
||||
this.expiryTimeMs = expiryTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Put an array of FileInputStream objects into the cache.
|
||||
*
|
||||
* @param datanodeID The DatanodeID to store the streams under.
|
||||
* @param block The Block to store the streams under.
|
||||
* @param fis The streams.
|
||||
*/
|
||||
public void put(DatanodeID datanodeID, ExtendedBlock block,
|
||||
FileInputStream fis[]) {
|
||||
boolean inserted = false;
|
||||
try {
|
||||
synchronized(this) {
|
||||
if (closed) return;
|
||||
if (map.size() + 1 > maxCacheSize) {
|
||||
Iterator<Entry<Key, Value>> iter = map.entries().iterator();
|
||||
if (!iter.hasNext()) return;
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
entry.getValue().close();
|
||||
iter.remove();
|
||||
}
|
||||
if (cacheCleaner == null) {
|
||||
cacheCleaner = new CacheCleaner();
|
||||
executor.scheduleAtFixedRate(cacheCleaner, expiryTimeMs, expiryTimeMs,
|
||||
TimeUnit.MILLISECONDS);
|
||||
}
|
||||
map.put(new Key(datanodeID, block), new Value(fis));
|
||||
inserted = true;
|
||||
}
|
||||
} finally {
|
||||
if (!inserted) {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Find and remove an array of FileInputStream objects from the cache.
|
||||
*
|
||||
* @param datanodeID The DatanodeID to search for.
|
||||
* @param block The Block to search for.
|
||||
*
|
||||
* @return null if no streams can be found; the
|
||||
* array otherwise. If this is non-null, the
|
||||
* array will have been removed from the cache.
|
||||
*/
|
||||
public synchronized FileInputStream[] get(DatanodeID datanodeID,
|
||||
ExtendedBlock block) {
|
||||
Key key = new Key(datanodeID, block);
|
||||
List<Value> ret = map.get(key);
|
||||
if (ret.isEmpty()) return null;
|
||||
Value val = ret.get(0);
|
||||
map.remove(key, val);
|
||||
return val.getFileInputStreams();
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the cache and free all associated resources.
|
||||
*/
|
||||
public synchronized void close() {
|
||||
if (closed) return;
|
||||
closed = true;
|
||||
if (cacheCleaner != null) {
|
||||
executor.remove(cacheCleaner);
|
||||
}
|
||||
for (Iterator<Entry<Key, Value>> iter = map.entries().iterator();
|
||||
iter.hasNext();
|
||||
iter = map.entries().iterator()) {
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
entry.getValue().close();
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized String toString() {
|
||||
StringBuilder bld = new StringBuilder();
|
||||
bld.append("FileInputStreamCache(");
|
||||
String prefix = "";
|
||||
for (Entry<Key, Value> entry : map.entries()) {
|
||||
bld.append(prefix);
|
||||
bld.append(entry.getKey());
|
||||
prefix = ", ";
|
||||
}
|
||||
bld.append(")");
|
||||
return bld.toString();
|
||||
}
|
||||
|
||||
public long getExpiryTimeMs() {
|
||||
return expiryTimeMs;
|
||||
}
|
||||
|
||||
public int getMaxCacheSize() {
|
||||
return maxCacheSize;
|
||||
}
|
||||
}
|
|
@ -18,92 +18,104 @@
|
|||
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketAddress;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import java.io.IOException;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.LinkedListMultimap;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
/**
|
||||
* A cache of input stream sockets to Data Node.
|
||||
*/
|
||||
class SocketCache {
|
||||
private static final Log LOG = LogFactory.getLog(SocketCache.class);
|
||||
|
||||
@InterfaceAudience.Private
|
||||
static class SocketAndStreams implements Closeable {
|
||||
public final Socket sock;
|
||||
public final IOStreamPair ioStreams;
|
||||
long createTime;
|
||||
class PeerCache {
|
||||
private static final Log LOG = LogFactory.getLog(PeerCache.class);
|
||||
|
||||
private static class Key {
|
||||
final DatanodeID dnID;
|
||||
final boolean isDomain;
|
||||
|
||||
public SocketAndStreams(Socket s, IOStreamPair ioStreams) {
|
||||
this.sock = s;
|
||||
this.ioStreams = ioStreams;
|
||||
this.createTime = Time.monotonicNow();
|
||||
Key(DatanodeID dnID, boolean isDomain) {
|
||||
this.dnID = dnID;
|
||||
this.isDomain = isDomain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (ioStreams != null) {
|
||||
IOUtils.closeStream(ioStreams.in);
|
||||
IOUtils.closeStream(ioStreams.out);
|
||||
public boolean equals(Object o) {
|
||||
if (!(o instanceof Key)) {
|
||||
return false;
|
||||
}
|
||||
IOUtils.closeSocket(sock);
|
||||
Key other = (Key)o;
|
||||
return dnID.equals(other.dnID) && isDomain == other.isDomain;
|
||||
}
|
||||
|
||||
public long getCreateTime() {
|
||||
return this.createTime;
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return dnID.hashCode() ^ (isDomain ? 1 : 0);
|
||||
}
|
||||
}
|
||||
|
||||
private static class Value {
|
||||
private final Peer peer;
|
||||
private final long time;
|
||||
|
||||
Value(Peer peer, long time) {
|
||||
this.peer = peer;
|
||||
this.time = time;
|
||||
}
|
||||
|
||||
Peer getPeer() {
|
||||
return peer;
|
||||
}
|
||||
|
||||
long getTime() {
|
||||
return time;
|
||||
}
|
||||
}
|
||||
|
||||
private Daemon daemon;
|
||||
/** A map for per user per datanode. */
|
||||
private static LinkedListMultimap<SocketAddress, SocketAndStreams> multimap =
|
||||
private final LinkedListMultimap<Key, Value> multimap =
|
||||
LinkedListMultimap.create();
|
||||
private static int capacity;
|
||||
private static long expiryPeriod;
|
||||
private static SocketCache scInstance = new SocketCache();
|
||||
private static boolean isInitedOnce = false;
|
||||
|
||||
public static synchronized SocketCache getInstance(int c, long e) {
|
||||
// capacity is only initialized once
|
||||
if (isInitedOnce == false) {
|
||||
capacity = c;
|
||||
expiryPeriod = e;
|
||||
private final int capacity;
|
||||
private final long expiryPeriod;
|
||||
private static PeerCache instance = null;
|
||||
|
||||
@VisibleForTesting
|
||||
PeerCache(int c, long e) {
|
||||
this.capacity = c;
|
||||
this.expiryPeriod = e;
|
||||
|
||||
if (capacity == 0 ) {
|
||||
LOG.info("SocketCache disabled.");
|
||||
}
|
||||
else if (expiryPeriod == 0) {
|
||||
throw new IllegalStateException("Cannot initialize expiryPeriod to " +
|
||||
expiryPeriod + "when cache is enabled.");
|
||||
}
|
||||
isInitedOnce = true;
|
||||
if (capacity == 0 ) {
|
||||
LOG.info("SocketCache disabled.");
|
||||
}
|
||||
else if (expiryPeriod == 0) {
|
||||
throw new IllegalStateException("Cannot initialize expiryPeriod to " +
|
||||
expiryPeriod + "when cache is enabled.");
|
||||
}
|
||||
}
|
||||
|
||||
public static synchronized PeerCache getInstance(int c, long e) {
|
||||
// capacity is only initialized once
|
||||
if (instance == null) {
|
||||
instance = new PeerCache(c, e);
|
||||
} else { //already initialized once
|
||||
if (capacity != c || expiryPeriod != e) {
|
||||
LOG.info("capacity and expiry periods already set to " + capacity +
|
||||
" and " + expiryPeriod + " respectively. Cannot set it to " + c +
|
||||
" and " + e);
|
||||
if (instance.capacity != c || instance.expiryPeriod != e) {
|
||||
LOG.info("capacity and expiry periods already set to " +
|
||||
instance.capacity + " and " + instance.expiryPeriod +
|
||||
" respectively. Cannot set it to " + c + " and " + e);
|
||||
}
|
||||
}
|
||||
|
||||
return scInstance;
|
||||
return instance;
|
||||
}
|
||||
|
||||
private boolean isDaemonStarted() {
|
||||
|
@ -120,44 +132,47 @@ class SocketCache {
|
|||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
SocketCache.this.run();
|
||||
PeerCache.this.run();
|
||||
} catch(InterruptedException e) {
|
||||
//noop
|
||||
} finally {
|
||||
SocketCache.this.clear();
|
||||
PeerCache.this.clear();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.valueOf(SocketCache.this);
|
||||
return String.valueOf(PeerCache.this);
|
||||
}
|
||||
});
|
||||
daemon.start();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a cached socket to the given address.
|
||||
* @param remote Remote address the socket is connected to.
|
||||
* @return A socket with unknown state, possibly closed underneath. Or null.
|
||||
* Get a cached peer connected to the given DataNode.
|
||||
* @param dnId The DataNode to get a Peer for.
|
||||
* @param isDomain Whether to retrieve a DomainPeer or not.
|
||||
*
|
||||
* @return An open Peer connected to the DN, or null if none
|
||||
* was found.
|
||||
*/
|
||||
public synchronized SocketAndStreams get(SocketAddress remote) {
|
||||
public synchronized Peer get(DatanodeID dnId, boolean isDomain) {
|
||||
|
||||
if (capacity <= 0) { // disabled
|
||||
return null;
|
||||
}
|
||||
|
||||
List<SocketAndStreams> sockStreamList = multimap.get(remote);
|
||||
List<Value> sockStreamList = multimap.get(new Key(dnId, isDomain));
|
||||
if (sockStreamList == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Iterator<SocketAndStreams> iter = sockStreamList.iterator();
|
||||
Iterator<Value> iter = sockStreamList.iterator();
|
||||
while (iter.hasNext()) {
|
||||
SocketAndStreams candidate = iter.next();
|
||||
Value candidate = iter.next();
|
||||
iter.remove();
|
||||
if (!candidate.sock.isClosed()) {
|
||||
return candidate;
|
||||
if (!candidate.getPeer().isClosed()) {
|
||||
return candidate.getPeer();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -167,30 +182,23 @@ class SocketCache {
|
|||
* Give an unused socket to the cache.
|
||||
* @param sock socket not used by anyone.
|
||||
*/
|
||||
public synchronized void put(Socket sock, IOStreamPair ioStreams) {
|
||||
|
||||
Preconditions.checkNotNull(sock);
|
||||
SocketAndStreams s = new SocketAndStreams(sock, ioStreams);
|
||||
public synchronized void put(DatanodeID dnId, Peer peer) {
|
||||
Preconditions.checkNotNull(dnId);
|
||||
Preconditions.checkNotNull(peer);
|
||||
if (peer.isClosed()) return;
|
||||
if (capacity <= 0) {
|
||||
// Cache disabled.
|
||||
s.close();
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
return;
|
||||
}
|
||||
|
||||
startExpiryDaemon();
|
||||
|
||||
SocketAddress remoteAddr = sock.getRemoteSocketAddress();
|
||||
if (remoteAddr == null) {
|
||||
LOG.warn("Cannot cache (unconnected) socket with no remote address: " +
|
||||
sock);
|
||||
IOUtils.closeSocket(sock);
|
||||
return;
|
||||
}
|
||||
|
||||
if (capacity == multimap.size()) {
|
||||
evictOldest();
|
||||
}
|
||||
multimap.put(remoteAddr, s);
|
||||
multimap.put(new Key(dnId, peer.getDomainSocket() != null),
|
||||
new Value(peer, Time.monotonicNow()));
|
||||
}
|
||||
|
||||
public synchronized int size() {
|
||||
|
@ -202,18 +210,17 @@ class SocketCache {
|
|||
*/
|
||||
private synchronized void evictExpired(long expiryPeriod) {
|
||||
while (multimap.size() != 0) {
|
||||
Iterator<Entry<SocketAddress, SocketAndStreams>> iter =
|
||||
Iterator<Entry<Key, Value>> iter =
|
||||
multimap.entries().iterator();
|
||||
Entry<SocketAddress, SocketAndStreams> entry = iter.next();
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
// if oldest socket expired, remove it
|
||||
if (entry == null ||
|
||||
Time.monotonicNow() - entry.getValue().getCreateTime() <
|
||||
Time.monotonicNow() - entry.getValue().getTime() <
|
||||
expiryPeriod) {
|
||||
break;
|
||||
}
|
||||
IOUtils.cleanup(LOG, entry.getValue().getPeer());
|
||||
iter.remove();
|
||||
SocketAndStreams s = entry.getValue();
|
||||
s.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -221,16 +228,18 @@ class SocketCache {
|
|||
* Evict the oldest entry in the cache.
|
||||
*/
|
||||
private synchronized void evictOldest() {
|
||||
Iterator<Entry<SocketAddress, SocketAndStreams>> iter =
|
||||
// We can get the oldest element immediately, because of an interesting
|
||||
// property of LinkedListMultimap: its iterator traverses entries in the
|
||||
// order that they were added.
|
||||
Iterator<Entry<Key, Value>> iter =
|
||||
multimap.entries().iterator();
|
||||
if (!iter.hasNext()) {
|
||||
throw new IllegalStateException("Cannot evict from empty cache! " +
|
||||
"capacity: " + capacity);
|
||||
}
|
||||
Entry<SocketAddress, SocketAndStreams> entry = iter.next();
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
IOUtils.cleanup(LOG, entry.getValue().getPeer());
|
||||
iter.remove();
|
||||
SocketAndStreams s = entry.getValue();
|
||||
s.close();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -255,11 +264,24 @@ class SocketCache {
|
|||
* Empty the cache, and close all sockets.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected synchronized void clear() {
|
||||
for (SocketAndStreams sockAndStream : multimap.values()) {
|
||||
sockAndStream.close();
|
||||
synchronized void clear() {
|
||||
for (Value value : multimap.values()) {
|
||||
IOUtils.cleanup(LOG, value.getPeer());
|
||||
}
|
||||
multimap.clear();
|
||||
}
|
||||
|
||||
|
||||
@VisibleForTesting
|
||||
void close() {
|
||||
clear();
|
||||
if (daemon != null) {
|
||||
daemon.interrupt();
|
||||
try {
|
||||
daemon.join();
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException("failed to join thread");
|
||||
}
|
||||
}
|
||||
daemon = null;
|
||||
}
|
||||
}
|
|
@ -29,6 +29,8 @@ 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.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
|
@ -54,8 +56,8 @@ import org.apache.hadoop.util.DataChecksum;
|
|||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
||||
|
||||
Socket dnSock; //for now just sending the status code (e.g. checksumOk) after the read.
|
||||
private final Peer peer;
|
||||
private final DatanodeID datanodeID;
|
||||
private final DataInputStream in;
|
||||
private DataChecksum checksum;
|
||||
|
||||
|
@ -125,9 +127,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
// if eos was set in the previous read, send a status code to the DN
|
||||
if (eos && !eosBefore && nRead >= 0) {
|
||||
if (needChecksum()) {
|
||||
sendReadResult(dnSock, Status.CHECKSUM_OK);
|
||||
sendReadResult(peer, Status.CHECKSUM_OK);
|
||||
} else {
|
||||
sendReadResult(dnSock, Status.SUCCESS);
|
||||
sendReadResult(peer, Status.SUCCESS);
|
||||
}
|
||||
}
|
||||
return nRead;
|
||||
|
@ -321,7 +323,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
|
||||
private RemoteBlockReader(String file, String bpid, long blockId,
|
||||
DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
|
||||
long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock) {
|
||||
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
|
||||
DatanodeID datanodeID) {
|
||||
// Path is used only for printing block and file information in debug
|
||||
super(new Path("/blk_" + blockId + ":" + bpid + ":of:"+ file)/*too non path-like?*/,
|
||||
1, verifyChecksum,
|
||||
|
@ -329,7 +332,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
checksum.getBytesPerChecksum(),
|
||||
checksum.getChecksumSize());
|
||||
|
||||
this.dnSock = dnSock;
|
||||
this.peer = peer;
|
||||
this.datanodeID = datanodeID;
|
||||
this.in = in;
|
||||
this.checksum = checksum;
|
||||
this.startOffset = Math.max( startOffset, 0 );
|
||||
|
@ -348,13 +352,6 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
checksumSize = this.checksum.getChecksumSize();
|
||||
}
|
||||
|
||||
public static RemoteBlockReader newBlockReader(Socket sock, String file,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len, int bufferSize) throws IOException {
|
||||
return newBlockReader(sock, file, block, blockToken, startOffset,
|
||||
len, bufferSize, true, "");
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new BlockReader specifically to satisfy a read.
|
||||
* This method also sends the OP_READ_BLOCK request.
|
||||
|
@ -370,16 +367,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
* @param clientName Client name
|
||||
* @return New BlockReader instance, or null on error.
|
||||
*/
|
||||
public static RemoteBlockReader newBlockReader( Socket sock, String file,
|
||||
public static RemoteBlockReader newBlockReader(String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
int bufferSize, boolean verifyChecksum,
|
||||
String clientName)
|
||||
String clientName, Peer peer,
|
||||
DatanodeID datanodeID)
|
||||
throws IOException {
|
||||
// in and out will be closed when sock is closed (by the caller)
|
||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT)));
|
||||
final DataOutputStream out =
|
||||
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
|
||||
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
|
||||
verifyChecksum);
|
||||
|
||||
|
@ -388,12 +386,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
//
|
||||
|
||||
DataInputStream in = new DataInputStream(
|
||||
new BufferedInputStream(NetUtils.getInputStream(sock),
|
||||
bufferSize));
|
||||
new BufferedInputStream(peer.getInputStream(), bufferSize));
|
||||
|
||||
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
|
||||
PBHelper.vintPrefixed(in));
|
||||
RemoteBlockReader2.checkSuccess(status, sock, block, file);
|
||||
RemoteBlockReader2.checkSuccess(status, peer, block, file);
|
||||
ReadOpChecksumInfoProto checksumInfo =
|
||||
status.getReadOpChecksumInfo();
|
||||
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
||||
|
@ -411,15 +408,19 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
}
|
||||
|
||||
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
|
||||
in, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock);
|
||||
in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
|
||||
peer, datanodeID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
public synchronized void close(PeerCache peerCache,
|
||||
FileInputStreamCache fisCache) throws IOException {
|
||||
startOffset = -1;
|
||||
checksum = null;
|
||||
if (dnSock != null) {
|
||||
dnSock.close();
|
||||
if (peerCache != null & sentStatusCode) {
|
||||
peerCache.put(datanodeID, peer);
|
||||
} else {
|
||||
peer.close();
|
||||
}
|
||||
|
||||
// in will be closed when its Socket is closed.
|
||||
|
@ -436,37 +437,21 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
return readFully(this, buf, offset, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Socket takeSocket() {
|
||||
assert hasSentStatusCode() :
|
||||
"BlockReader shouldn't give back sockets mid-read";
|
||||
Socket res = dnSock;
|
||||
dnSock = null;
|
||||
return res;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasSentStatusCode() {
|
||||
return sentStatusCode;
|
||||
}
|
||||
|
||||
/**
|
||||
* When the reader reaches end of the read, it sends a status response
|
||||
* (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
|
||||
* closing our connection (which we will re-open), but won't affect
|
||||
* data correctness.
|
||||
*/
|
||||
void sendReadResult(Socket sock, Status statusCode) {
|
||||
assert !sentStatusCode : "already sent status code to " + sock;
|
||||
void sendReadResult(Peer peer, Status statusCode) {
|
||||
assert !sentStatusCode : "already sent status code to " + peer;
|
||||
try {
|
||||
RemoteBlockReader2.writeReadResult(
|
||||
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT),
|
||||
statusCode);
|
||||
RemoteBlockReader2.writeReadResult(peer.getOutputStream(), statusCode);
|
||||
sentStatusCode = true;
|
||||
} catch (IOException e) {
|
||||
// It's ok not to be able to send this. But something is probably wrong.
|
||||
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
|
||||
sock.getInetAddress() + ": " + e.getMessage());
|
||||
peer.getRemoteAddressString() + ": " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -486,12 +471,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
public int read(ByteBuffer buf) throws IOException {
|
||||
throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public IOStreamPair getStreams() {
|
||||
// This class doesn't support encryption, which is the only thing this
|
||||
// method is used for. See HDFS-3637.
|
||||
return null;
|
||||
public int available() throws IOException {
|
||||
// An optimistic estimate of how much data is available
|
||||
// to us without doing network I/O.
|
||||
return DFSClient.TCP_WINDOW_SIZE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -23,16 +23,16 @@ import java.io.DataOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.nio.ByteBuffer;
|
||||
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.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
||||
|
@ -44,10 +44,11 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||
import org.apache.hadoop.net.SocketInputWrapper;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* This is a wrapper around connection to datanode
|
||||
* and understands checksum, offset etc.
|
||||
|
@ -79,9 +80,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
|
||||
static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class);
|
||||
|
||||
Socket dnSock;
|
||||
// for now just sending the status code (e.g. checksumOk) after the read.
|
||||
private IOStreamPair ioStreams;
|
||||
final private Peer peer;
|
||||
final private DatanodeID datanodeID;
|
||||
private final ReadableByteChannel in;
|
||||
private DataChecksum checksum;
|
||||
|
||||
|
@ -114,6 +114,11 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
/** Amount of unread data in the current received packet */
|
||||
int dataLeft = 0;
|
||||
|
||||
@VisibleForTesting
|
||||
public Peer getPeer() {
|
||||
return peer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(byte[] buf, int off, int len)
|
||||
throws IOException {
|
||||
|
@ -246,13 +251,13 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
}
|
||||
|
||||
protected RemoteBlockReader2(String file, String bpid, long blockId,
|
||||
ReadableByteChannel in, DataChecksum checksum, boolean verifyChecksum,
|
||||
long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock,
|
||||
IOStreamPair ioStreams) {
|
||||
DataChecksum checksum, boolean verifyChecksum,
|
||||
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
|
||||
DatanodeID datanodeID) {
|
||||
// Path is used only for printing block and file information in debug
|
||||
this.dnSock = dnSock;
|
||||
this.ioStreams = ioStreams;
|
||||
this.in = in;
|
||||
this.peer = peer;
|
||||
this.datanodeID = datanodeID;
|
||||
this.in = peer.getInputStreamChannel();
|
||||
this.checksum = checksum;
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
this.startOffset = Math.max( startOffset, 0 );
|
||||
|
@ -269,39 +274,20 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
public synchronized void close(PeerCache peerCache,
|
||||
FileInputStreamCache fisCache) throws IOException {
|
||||
packetReceiver.close();
|
||||
|
||||
startOffset = -1;
|
||||
checksum = null;
|
||||
if (dnSock != null) {
|
||||
dnSock.close();
|
||||
if (peerCache != null && sentStatusCode) {
|
||||
peerCache.put(datanodeID, peer);
|
||||
} else {
|
||||
peer.close();
|
||||
}
|
||||
|
||||
// in will be closed when its Socket is closed.
|
||||
}
|
||||
|
||||
/**
|
||||
* Take the socket used to talk to the DN.
|
||||
*/
|
||||
@Override
|
||||
public Socket takeSocket() {
|
||||
assert hasSentStatusCode() :
|
||||
"BlockReader shouldn't give back sockets mid-read";
|
||||
Socket res = dnSock;
|
||||
dnSock = null;
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether the BlockReader has reached the end of its input stream
|
||||
* and successfully sent a status code back to the datanode.
|
||||
*/
|
||||
@Override
|
||||
public boolean hasSentStatusCode() {
|
||||
return sentStatusCode;
|
||||
}
|
||||
|
||||
/**
|
||||
* When the reader reaches end of the read, it sends a status response
|
||||
* (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
|
||||
|
@ -309,14 +295,14 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
* data correctness.
|
||||
*/
|
||||
void sendReadResult(Status statusCode) {
|
||||
assert !sentStatusCode : "already sent status code to " + dnSock;
|
||||
assert !sentStatusCode : "already sent status code to " + peer;
|
||||
try {
|
||||
writeReadResult(ioStreams.out, statusCode);
|
||||
writeReadResult(peer.getOutputStream(), statusCode);
|
||||
sentStatusCode = true;
|
||||
} catch (IOException e) {
|
||||
// It's ok not to be able to send this. But something is probably wrong.
|
||||
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
|
||||
dnSock.getInetAddress() + ": " + e.getMessage());
|
||||
peer.getRemoteAddressString() + ": " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -367,42 +353,34 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
* @param blockToken The block token for security
|
||||
* @param startOffset The read offset, relative to block head
|
||||
* @param len The number of bytes to read
|
||||
* @param bufferSize The IO buffer size (not the client buffer size)
|
||||
* @param verifyChecksum Whether to verify checksum
|
||||
* @param clientName Client name
|
||||
* @param peer The Peer to use
|
||||
* @param datanodeID The DatanodeID this peer is connected to
|
||||
* @return New BlockReader instance, or null on error.
|
||||
*/
|
||||
public static BlockReader newBlockReader(Socket sock, String file,
|
||||
public static BlockReader newBlockReader(String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
int bufferSize, boolean verifyChecksum,
|
||||
boolean verifyChecksum,
|
||||
String clientName,
|
||||
DataEncryptionKey encryptionKey,
|
||||
IOStreamPair ioStreams)
|
||||
Peer peer, DatanodeID datanodeID)
|
||||
throws IOException {
|
||||
|
||||
ReadableByteChannel ch;
|
||||
if (ioStreams.in instanceof SocketInputWrapper) {
|
||||
ch = ((SocketInputWrapper)ioStreams.in).getReadableByteChannel();
|
||||
} else {
|
||||
ch = (ReadableByteChannel) ioStreams.in;
|
||||
}
|
||||
|
||||
// in and out will be closed when sock is closed (by the caller)
|
||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||
ioStreams.out));
|
||||
peer.getOutputStream()));
|
||||
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
|
||||
verifyChecksum);
|
||||
|
||||
//
|
||||
// Get bytes in block
|
||||
//
|
||||
DataInputStream in = new DataInputStream(ioStreams.in);
|
||||
DataInputStream in = new DataInputStream(peer.getInputStream());
|
||||
|
||||
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
|
||||
PBHelper.vintPrefixed(in));
|
||||
checkSuccess(status, sock, block, file);
|
||||
checkSuccess(status, peer, block, file);
|
||||
ReadOpChecksumInfoProto checksumInfo =
|
||||
status.getReadOpChecksumInfo();
|
||||
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
||||
|
@ -420,34 +398,36 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
}
|
||||
|
||||
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
|
||||
ch, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock,
|
||||
ioStreams);
|
||||
checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
|
||||
datanodeID);
|
||||
}
|
||||
|
||||
static void checkSuccess(
|
||||
BlockOpResponseProto status, Socket sock,
|
||||
BlockOpResponseProto status, Peer peer,
|
||||
ExtendedBlock block, String file)
|
||||
throws IOException {
|
||||
if (status.getStatus() != Status.SUCCESS) {
|
||||
if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) {
|
||||
throw new InvalidBlockTokenException(
|
||||
"Got access token error for OP_READ_BLOCK, self="
|
||||
+ sock.getLocalSocketAddress() + ", remote="
|
||||
+ sock.getRemoteSocketAddress() + ", for file " + file
|
||||
+ peer.getLocalAddressString() + ", remote="
|
||||
+ peer.getRemoteAddressString() + ", for file " + file
|
||||
+ ", for pool " + block.getBlockPoolId() + " block "
|
||||
+ block.getBlockId() + "_" + block.getGenerationStamp());
|
||||
} else {
|
||||
throw new IOException("Got error for OP_READ_BLOCK, self="
|
||||
+ sock.getLocalSocketAddress() + ", remote="
|
||||
+ sock.getRemoteSocketAddress() + ", for file " + file
|
||||
+ peer.getLocalAddressString() + ", remote="
|
||||
+ peer.getRemoteAddressString() + ", for file " + file
|
||||
+ ", for pool " + block.getBlockPoolId() + " block "
|
||||
+ block.getBlockId() + "_" + block.getGenerationStamp());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public IOStreamPair getStreams() {
|
||||
return ioStreams;
|
||||
public int available() throws IOException {
|
||||
// An optimistic estimate of how much data is available
|
||||
// to us without doing network I/O.
|
||||
return DFSClient.TCP_WINDOW_SIZE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,128 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.Socket;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
|
||||
/**
|
||||
* Represents a peer that we communicate with by using a basic Socket
|
||||
* that has no associated Channel.
|
||||
*
|
||||
*/
|
||||
class BasicInetPeer implements Peer {
|
||||
private final Socket socket;
|
||||
private final OutputStream out;
|
||||
private final InputStream in;
|
||||
private final boolean isLocal;
|
||||
|
||||
public BasicInetPeer(Socket socket) throws IOException {
|
||||
this.socket = socket;
|
||||
this.out = socket.getOutputStream();
|
||||
this.in = socket.getInputStream();
|
||||
this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableByteChannel getInputStreamChannel() {
|
||||
/*
|
||||
* This Socket has no channel, so there's nothing to return here.
|
||||
*/
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReadTimeout(int timeoutMs) throws IOException {
|
||||
socket.setSoTimeout(timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getReceiveBufferSize() throws IOException {
|
||||
return socket.getReceiveBufferSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getTcpNoDelay() throws IOException {
|
||||
return socket.getTcpNoDelay();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteTimeout(int timeoutMs) {
|
||||
/*
|
||||
* We can't implement write timeouts. :(
|
||||
*
|
||||
* Java provides no facility to set a blocking write timeout on a Socket.
|
||||
* You can simulate a blocking write with a timeout by using
|
||||
* non-blocking I/O. However, we can't use nio here, because this Socket
|
||||
* doesn't have an associated Channel.
|
||||
*
|
||||
* See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
|
||||
* more details.
|
||||
*/
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return socket.isClosed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
socket.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRemoteAddressString() {
|
||||
return socket.getRemoteSocketAddress().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLocalAddressString() {
|
||||
return socket.getLocalSocketAddress().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream() throws IOException {
|
||||
return in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream getOutputStream() throws IOException {
|
||||
return out;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocal() {
|
||||
return isLocal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "BasicInetPeer(" + socket.toString() + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DomainSocket getDomainSocket() {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Represents a peer that we communicate with by using blocking I/O
|
||||
* on a UNIX domain socket.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DomainPeer implements Peer {
|
||||
private final DomainSocket socket;
|
||||
private final OutputStream out;
|
||||
private final InputStream in;
|
||||
private final ReadableByteChannel channel;
|
||||
|
||||
public DomainPeer(DomainSocket socket) {
|
||||
this.socket = socket;
|
||||
this.out = socket.getOutputStream();
|
||||
this.in = socket.getInputStream();
|
||||
this.channel = socket.getChannel();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableByteChannel getInputStreamChannel() {
|
||||
return channel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReadTimeout(int timeoutMs) throws IOException {
|
||||
socket.setAttribute(DomainSocket.RECEIVE_TIMEOUT, timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getReceiveBufferSize() throws IOException {
|
||||
return socket.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getTcpNoDelay() throws IOException {
|
||||
/* No TCP, no TCP_NODELAY. */
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteTimeout(int timeoutMs) throws IOException {
|
||||
socket.setAttribute(DomainSocket.SEND_TIMEOUT, timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return !socket.isOpen();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
socket.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRemoteAddressString() {
|
||||
return "unix:" + socket.getPath();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLocalAddressString() {
|
||||
return "<local>";
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream() throws IOException {
|
||||
return in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream getOutputStream() throws IOException {
|
||||
return out;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocal() {
|
||||
/* UNIX domain sockets can only be used for local communication. */
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DomainPeer(" + getRemoteAddressString() + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DomainSocket getDomainSocket() {
|
||||
return socket;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.SocketTimeoutException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.hdfs.net.PeerServer;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class DomainPeerServer implements PeerServer {
|
||||
static Log LOG = LogFactory.getLog(DomainPeerServer.class);
|
||||
private final DomainSocket sock;
|
||||
|
||||
DomainPeerServer(DomainSocket sock) {
|
||||
this.sock = sock;
|
||||
}
|
||||
|
||||
public DomainPeerServer(String path, int port)
|
||||
throws IOException {
|
||||
this(DomainSocket.bindAndListen(DomainSocket.getEffectivePath(path, port)));
|
||||
}
|
||||
|
||||
public String getBindPath() {
|
||||
return sock.getPath();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReceiveBufferSize(int size) throws IOException {
|
||||
sock.setAttribute(DomainSocket.RECEIVE_BUFFER_SIZE, size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Peer accept() throws IOException, SocketTimeoutException {
|
||||
DomainSocket connSock = sock.accept();
|
||||
Peer peer = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
peer = new DomainPeer(connSock);
|
||||
success = true;
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (peer != null) peer.close();
|
||||
connSock.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getListeningString() {
|
||||
return "unix:" + sock.getPath();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
sock.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("error closing DomainPeerServer: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DomainPeerServer(" + getListeningString() + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,142 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
|
||||
/**
|
||||
* Represents a peer that we communicate with by using an encrypted
|
||||
* communications medium.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class EncryptedPeer implements Peer {
|
||||
private final Peer enclosedPeer;
|
||||
|
||||
/**
|
||||
* An encrypted InputStream.
|
||||
*/
|
||||
private final InputStream in;
|
||||
|
||||
/**
|
||||
* An encrypted OutputStream.
|
||||
*/
|
||||
private final OutputStream out;
|
||||
|
||||
/**
|
||||
* An encrypted ReadableByteChannel.
|
||||
*/
|
||||
private final ReadableByteChannel channel;
|
||||
|
||||
public EncryptedPeer(Peer enclosedPeer, DataEncryptionKey key)
|
||||
throws IOException {
|
||||
this.enclosedPeer = enclosedPeer;
|
||||
IOStreamPair ios = DataTransferEncryptor.getEncryptedStreams(
|
||||
enclosedPeer.getOutputStream(), enclosedPeer.getInputStream(), key);
|
||||
this.in = ios.in;
|
||||
this.out = ios.out;
|
||||
this.channel = ios.in instanceof ReadableByteChannel ?
|
||||
(ReadableByteChannel)ios.in : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableByteChannel getInputStreamChannel() {
|
||||
return channel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReadTimeout(int timeoutMs) throws IOException {
|
||||
enclosedPeer.setReadTimeout(timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getReceiveBufferSize() throws IOException {
|
||||
return enclosedPeer.getReceiveBufferSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getTcpNoDelay() throws IOException {
|
||||
return enclosedPeer.getTcpNoDelay();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteTimeout(int timeoutMs) throws IOException {
|
||||
enclosedPeer.setWriteTimeout(timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return enclosedPeer.isClosed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
in.close();
|
||||
} finally {
|
||||
try {
|
||||
out.close();
|
||||
} finally {
|
||||
enclosedPeer.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRemoteAddressString() {
|
||||
return enclosedPeer.getRemoteAddressString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLocalAddressString() {
|
||||
return enclosedPeer.getLocalAddressString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream() throws IOException {
|
||||
return in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream getOutputStream() throws IOException {
|
||||
return out;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocal() {
|
||||
return enclosedPeer.isLocal();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "EncryptedPeer(" + enclosedPeer + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DomainSocket getDomainSocket() {
|
||||
return enclosedPeer.getDomainSocket();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,131 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.Socket;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
|
||||
import org.apache.hadoop.net.SocketInputStream;
|
||||
import org.apache.hadoop.net.SocketOutputStream;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
|
||||
/**
|
||||
* Represents a peer that we communicate with by using non-blocking I/O
|
||||
* on a Socket.
|
||||
*/
|
||||
class NioInetPeer implements Peer {
|
||||
private final Socket socket;
|
||||
|
||||
/**
|
||||
* An InputStream which simulates blocking I/O with timeouts using NIO.
|
||||
*/
|
||||
private final SocketInputStream in;
|
||||
|
||||
/**
|
||||
* An OutputStream which simulates blocking I/O with timeouts using NIO.
|
||||
*/
|
||||
private final SocketOutputStream out;
|
||||
|
||||
private final boolean isLocal;
|
||||
|
||||
NioInetPeer(Socket socket) throws IOException {
|
||||
this.socket = socket;
|
||||
this.in = new SocketInputStream(socket.getChannel(), 0);
|
||||
this.out = new SocketOutputStream(socket.getChannel(), 0);
|
||||
this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableByteChannel getInputStreamChannel() {
|
||||
return in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReadTimeout(int timeoutMs) throws IOException {
|
||||
in.setTimeout(timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getReceiveBufferSize() throws IOException {
|
||||
return socket.getReceiveBufferSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getTcpNoDelay() throws IOException {
|
||||
return socket.getTcpNoDelay();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteTimeout(int timeoutMs) throws IOException {
|
||||
out.setTimeout(timeoutMs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return socket.isClosed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// We always close the outermost streams-- in this case, 'in' and 'out'
|
||||
// Closing either one of these will also close the Socket.
|
||||
try {
|
||||
in.close();
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRemoteAddressString() {
|
||||
return socket.getRemoteSocketAddress().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLocalAddressString() {
|
||||
return socket.getLocalSocketAddress().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream() throws IOException {
|
||||
return in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream getOutputStream() throws IOException {
|
||||
return out;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocal() {
|
||||
return isLocal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "NioInetPeer(" + socket.toString() + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DomainSocket getDomainSocket() {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,115 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
|
||||
/**
|
||||
* Represents a connection to a peer.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface Peer extends Closeable {
|
||||
/**
|
||||
* @return The input stream channel associated with this
|
||||
* peer, or null if it has none.
|
||||
*/
|
||||
public ReadableByteChannel getInputStreamChannel();
|
||||
|
||||
/**
|
||||
* Set the read timeout on this peer.
|
||||
*
|
||||
* @param timeoutMs The timeout in milliseconds.
|
||||
*/
|
||||
public void setReadTimeout(int timeoutMs) throws IOException;
|
||||
|
||||
/**
|
||||
* @return The receive buffer size.
|
||||
*/
|
||||
public int getReceiveBufferSize() throws IOException;
|
||||
|
||||
/**
|
||||
* @return True if TCP_NODELAY is turned on.
|
||||
*/
|
||||
public boolean getTcpNoDelay() throws IOException;
|
||||
|
||||
/**
|
||||
* Set the write timeout on this peer.
|
||||
*
|
||||
* Note: this is not honored for BasicInetPeer.
|
||||
* See {@link BasicSocketPeer#setWriteTimeout} for details.
|
||||
*
|
||||
* @param timeoutMs The timeout in milliseconds.
|
||||
*/
|
||||
public void setWriteTimeout(int timeoutMs) throws IOException;
|
||||
|
||||
/**
|
||||
* @return true only if the peer is closed.
|
||||
*/
|
||||
public boolean isClosed();
|
||||
|
||||
/**
|
||||
* Close the peer.
|
||||
*
|
||||
* It's safe to re-close a Peer that is already closed.
|
||||
*/
|
||||
public void close() throws IOException;
|
||||
|
||||
/**
|
||||
* @return A string representing the remote end of our
|
||||
* connection to the peer.
|
||||
*/
|
||||
public String getRemoteAddressString();
|
||||
|
||||
/**
|
||||
* @return A string representing the local end of our
|
||||
* connection to the peer.
|
||||
*/
|
||||
public String getLocalAddressString();
|
||||
|
||||
/**
|
||||
* @return An InputStream associated with the Peer.
|
||||
* This InputStream will be valid until you close
|
||||
* this peer with Peer#close.
|
||||
*/
|
||||
public InputStream getInputStream() throws IOException;
|
||||
|
||||
/**
|
||||
* @return An OutputStream associated with the Peer.
|
||||
* This OutputStream will be valid until you close
|
||||
* this peer with Peer#close.
|
||||
*/
|
||||
public OutputStream getOutputStream() throws IOException;
|
||||
|
||||
/**
|
||||
* @return True if the peer resides on the same
|
||||
* computer as we.
|
||||
*/
|
||||
public boolean isLocal();
|
||||
|
||||
/**
|
||||
* @return The DomainSocket associated with the current
|
||||
* peer, or null if there is none.
|
||||
*/
|
||||
public DomainSocket getDomainSocket();
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.Closeable;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import java.io.IOException;
|
||||
import java.net.SocketTimeoutException;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public interface PeerServer extends Closeable {
|
||||
/**
|
||||
* Set the receive buffer size of the PeerServer.
|
||||
*
|
||||
* @param size The receive buffer size.
|
||||
*/
|
||||
public void setReceiveBufferSize(int size) throws IOException;
|
||||
|
||||
/**
|
||||
* Listens for a connection to be made to this server and accepts
|
||||
* it. The method blocks until a connection is made.
|
||||
*
|
||||
* @exception IOException if an I/O error occurs when waiting for a
|
||||
* connection.
|
||||
* @exception SecurityException if a security manager exists and its
|
||||
* <code>checkAccept</code> method doesn't allow the operation.
|
||||
* @exception SocketTimeoutException if a timeout was previously set and
|
||||
* the timeout has been reached.
|
||||
*/
|
||||
public Peer accept() throws IOException, SocketTimeoutException;
|
||||
|
||||
/**
|
||||
* @return A string representation of the address we're
|
||||
* listening on.
|
||||
*/
|
||||
public String getListeningString();
|
||||
|
||||
/**
|
||||
* Free the resources associated with this peer server.
|
||||
* This normally includes sockets, etc.
|
||||
*
|
||||
* @throws IOException If there is an error closing the PeerServer
|
||||
*/
|
||||
public void close() throws IOException;
|
||||
}
|
|
@ -0,0 +1,156 @@
|
|||
/**
|
||||
* 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.net;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.ipc.Server;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class TcpPeerServer implements PeerServer {
|
||||
static Log LOG = LogFactory.getLog(TcpPeerServer.class);
|
||||
|
||||
private final ServerSocket serverSocket;
|
||||
|
||||
public static Peer peerFromSocket(Socket socket)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
// TCP_NODELAY is crucial here because of bad interactions between
|
||||
// Nagle's Algorithm and Delayed ACKs. With connection keepalive
|
||||
// between the client and DN, the conversation looks like:
|
||||
// 1. Client -> DN: Read block X
|
||||
// 2. DN -> Client: data for block X
|
||||
// 3. Client -> DN: Status OK (successful read)
|
||||
// 4. Client -> DN: Read block Y
|
||||
// The fact that step #3 and #4 are both in the client->DN direction
|
||||
// triggers Nagling. If the DN is using delayed ACKs, this results
|
||||
// in a delay of 40ms or more.
|
||||
//
|
||||
// TCP_NODELAY disables nagling and thus avoids this performance
|
||||
// disaster.
|
||||
socket.setTcpNoDelay(true);
|
||||
SocketChannel channel = socket.getChannel();
|
||||
if (channel == null) {
|
||||
peer = new BasicInetPeer(socket);
|
||||
} else {
|
||||
peer = new NioInetPeer(socket);
|
||||
}
|
||||
success = true;
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (peer != null) peer.close();
|
||||
socket.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static Peer peerFromSocketAndKey(Socket s,
|
||||
DataEncryptionKey key) throws IOException {
|
||||
Peer peer = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
peer = peerFromSocket(s);
|
||||
if (key != null) {
|
||||
peer = new EncryptedPeer(peer, key);
|
||||
}
|
||||
success = true;
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.cleanup(null, peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a non-secure TcpPeerServer.
|
||||
*
|
||||
* @param socketWriteTimeout The Socket write timeout in ms.
|
||||
* @param bindAddr The address to bind to.
|
||||
* @throws IOException
|
||||
*/
|
||||
public TcpPeerServer(int socketWriteTimeout,
|
||||
InetSocketAddress bindAddr) throws IOException {
|
||||
this.serverSocket = (socketWriteTimeout > 0) ?
|
||||
ServerSocketChannel.open().socket() : new ServerSocket();
|
||||
Server.bind(serverSocket, bindAddr, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a secure TcpPeerServer.
|
||||
*
|
||||
* @param secureResources Security resources.
|
||||
*/
|
||||
public TcpPeerServer(SecureResources secureResources) {
|
||||
this.serverSocket = secureResources.getStreamingSocket();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the IP address which this TcpPeerServer is listening on.
|
||||
*/
|
||||
public InetSocketAddress getStreamingAddr() {
|
||||
return new InetSocketAddress(
|
||||
serverSocket.getInetAddress().getHostAddress(),
|
||||
serverSocket.getLocalPort());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReceiveBufferSize(int size) throws IOException {
|
||||
this.serverSocket.setReceiveBufferSize(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Peer accept() throws IOException, SocketTimeoutException {
|
||||
Peer peer = peerFromSocket(serverSocket.accept());
|
||||
return peer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getListeningString() {
|
||||
return serverSocket.getLocalSocketAddress().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
serverSocket.close();
|
||||
} catch(IOException e) {
|
||||
LOG.error("error closing TcpPeerServer: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "TcpPeerServer(" + getListeningString() + ")";
|
||||
}
|
||||
}
|
|
@ -107,6 +107,18 @@ public interface DataTransferProtocol {
|
|||
final String clientName,
|
||||
final DatanodeInfo[] targets) throws IOException;
|
||||
|
||||
/**
|
||||
* Request short circuit access file descriptors from a DataNode.
|
||||
*
|
||||
* @param blk The block to get file descriptors for.
|
||||
* @param blockToken Security token for accessing the block.
|
||||
* @param maxVersion Maximum version of the block data the client
|
||||
* can understand.
|
||||
*/
|
||||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
int maxVersion) throws IOException;
|
||||
|
||||
/**
|
||||
* Receive a block from a source datanode
|
||||
* and then notifies the namenode
|
||||
|
|
|
@ -34,7 +34,8 @@ public enum Op {
|
|||
REPLACE_BLOCK((byte)83),
|
||||
COPY_BLOCK((byte)84),
|
||||
BLOCK_CHECKSUM((byte)85),
|
||||
TRANSFER_BLOCK((byte)86);
|
||||
TRANSFER_BLOCK((byte)86),
|
||||
REQUEST_SHORT_CIRCUIT_FDS((byte)87);
|
||||
|
||||
/** The code for this operation. */
|
||||
public final byte code;
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto
|
|||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
|
||||
|
@ -76,6 +77,9 @@ public abstract class Receiver implements DataTransferProtocol {
|
|||
case TRANSFER_BLOCK:
|
||||
opTransferBlock(in);
|
||||
break;
|
||||
case REQUEST_SHORT_CIRCUIT_FDS:
|
||||
opRequestShortCircuitFds(in);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unknown op " + op + " in data stream");
|
||||
}
|
||||
|
@ -117,6 +121,15 @@ public abstract class Receiver implements DataTransferProtocol {
|
|||
PBHelper.convert(proto.getTargetsList()));
|
||||
}
|
||||
|
||||
/** Receive {@link Op#REQUEST_SHORT_CIRCUIT_FDS} */
|
||||
private void opRequestShortCircuitFds(DataInputStream in) throws IOException {
|
||||
final OpRequestShortCircuitAccessProto proto =
|
||||
OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in));
|
||||
requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
|
||||
PBHelper.convert(proto.getHeader().getToken()),
|
||||
proto.getMaxVersion());
|
||||
}
|
||||
|
||||
/** Receive OP_REPLACE_BLOCK */
|
||||
private void opReplaceBlock(DataInputStream in) throws IOException {
|
||||
OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto
|
|||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
|
@ -140,6 +141,17 @@ public class Sender implements DataTransferProtocol {
|
|||
send(out, Op.TRANSFER_BLOCK, proto);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
int maxVersion) throws IOException {
|
||||
OpRequestShortCircuitAccessProto proto =
|
||||
OpRequestShortCircuitAccessProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(
|
||||
blk, blockToken)).setMaxVersion(maxVersion).build();
|
||||
send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void replaceBlock(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
|
|
|
@ -45,6 +45,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
|
@ -208,9 +210,12 @@ public class JspHelper {
|
|||
// Use the block name for file name.
|
||||
String file = BlockReaderFactory.getFileName(addr, poolId, blockId);
|
||||
BlockReader blockReader = BlockReaderFactory.newBlockReader(
|
||||
conf, s, file,
|
||||
conf, file,
|
||||
new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken,
|
||||
offsetIntoBlock, amtToRead, encryptionKey);
|
||||
offsetIntoBlock, amtToRead, true,
|
||||
"JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey),
|
||||
new DatanodeID(addr.getAddress().toString(),
|
||||
addr.getHostName(), poolId, addr.getPort(), 0, 0), null, false);
|
||||
|
||||
byte[] buf = new byte[(int)amtToRead];
|
||||
int readOffset = 0;
|
||||
|
@ -229,8 +234,7 @@ public class JspHelper {
|
|||
amtToRead -= numRead;
|
||||
readOffset += numRead;
|
||||
}
|
||||
blockReader = null;
|
||||
s.close();
|
||||
blockReader.close(null, null);
|
||||
out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
|
||||
}
|
||||
|
||||
|
|
|
@ -53,19 +53,18 @@ import java.io.ByteArrayInputStream;
|
|||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.net.URI;
|
||||
import java.net.UnknownHostException;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.AbstractList;
|
||||
|
@ -93,6 +92,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeerServer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
|
@ -151,11 +152,11 @@ import org.apache.hadoop.io.ReadaheadPool;
|
|||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.metrics2.util.MBeans;
|
||||
import org.apache.hadoop.net.DNS;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
@ -235,6 +236,7 @@ public class DataNode extends Configured
|
|||
LogFactory.getLog(DataNode.class.getName() + ".clienttrace");
|
||||
|
||||
private static final String USAGE = "Usage: java DataNode [-rollback | -regular]";
|
||||
static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
|
||||
|
||||
/**
|
||||
* Use {@link NetUtils#createSocketAddr(String)} instead.
|
||||
|
@ -252,6 +254,7 @@ public class DataNode extends Configured
|
|||
public final static String EMPTY_DEL_HINT = "";
|
||||
AtomicInteger xmitsInProgress = new AtomicInteger();
|
||||
Daemon dataXceiverServer = null;
|
||||
Daemon localDataXceiverServer = null;
|
||||
ThreadGroup threadGroup = null;
|
||||
private DNConf dnConf;
|
||||
private volatile boolean heartbeatsDisabledForTests = false;
|
||||
|
@ -263,6 +266,7 @@ public class DataNode extends Configured
|
|||
private String hostName;
|
||||
private DatanodeID id;
|
||||
|
||||
final private String fileDescriptorPassingDisabledReason;
|
||||
boolean isBlockTokenEnabled;
|
||||
BlockPoolTokenSecretManager blockPoolTokenSecretManager;
|
||||
private boolean hasAnyBlockPoolRegistered = false;
|
||||
|
@ -311,6 +315,24 @@ public class DataNode extends Configured
|
|||
this.getHdfsBlockLocationsEnabled = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
|
||||
DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
|
||||
|
||||
// Determine whether we should try to pass file descriptors to clients.
|
||||
if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT)) {
|
||||
String reason = DomainSocket.getLoadingFailureReason();
|
||||
if (reason != null) {
|
||||
LOG.warn("File descriptor passing is disabled because " + reason);
|
||||
this.fileDescriptorPassingDisabledReason = reason;
|
||||
} else {
|
||||
LOG.info("File descriptor passing is enabled.");
|
||||
this.fileDescriptorPassingDisabledReason = null;
|
||||
}
|
||||
} else {
|
||||
this.fileDescriptorPassingDisabledReason =
|
||||
"File descriptor passing was not configured.";
|
||||
LOG.debug(this.fileDescriptorPassingDisabledReason);
|
||||
}
|
||||
|
||||
try {
|
||||
hostName = getHostName(conf);
|
||||
LOG.info("Configured hostname is " + hostName);
|
||||
|
@ -525,25 +547,63 @@ public class DataNode extends Configured
|
|||
|
||||
private void initDataXceiver(Configuration conf) throws IOException {
|
||||
// find free port or use privileged port provided
|
||||
ServerSocket ss;
|
||||
if (secureResources == null) {
|
||||
InetSocketAddress addr = DataNode.getStreamingAddr(conf);
|
||||
ss = (dnConf.socketWriteTimeout > 0) ?
|
||||
ServerSocketChannel.open().socket() : new ServerSocket();
|
||||
Server.bind(ss, addr, 0);
|
||||
TcpPeerServer tcpPeerServer;
|
||||
if (secureResources != null) {
|
||||
tcpPeerServer = new TcpPeerServer(secureResources);
|
||||
} else {
|
||||
ss = secureResources.getStreamingSocket();
|
||||
tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
|
||||
DataNode.getStreamingAddr(conf));
|
||||
}
|
||||
ss.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
|
||||
|
||||
streamingAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
|
||||
ss.getLocalPort());
|
||||
|
||||
tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
|
||||
streamingAddr = tcpPeerServer.getStreamingAddr();
|
||||
LOG.info("Opened streaming server at " + streamingAddr);
|
||||
this.threadGroup = new ThreadGroup("dataXceiverServer");
|
||||
this.dataXceiverServer = new Daemon(threadGroup,
|
||||
new DataXceiverServer(ss, conf, this));
|
||||
new DataXceiverServer(tcpPeerServer, conf, this));
|
||||
this.threadGroup.setDaemon(true); // auto destroy when empty
|
||||
|
||||
if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) ||
|
||||
conf.getBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
|
||||
DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
|
||||
DomainPeerServer domainPeerServer =
|
||||
getDomainPeerServer(conf, streamingAddr.getPort());
|
||||
if (domainPeerServer != null) {
|
||||
this.localDataXceiverServer = new Daemon(threadGroup,
|
||||
new DataXceiverServer(domainPeerServer, conf, this));
|
||||
LOG.info("Listening on UNIX domain socket: " +
|
||||
domainPeerServer.getBindPath());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static DomainPeerServer getDomainPeerServer(Configuration conf,
|
||||
int port) throws IOException {
|
||||
String domainSocketPath =
|
||||
conf.getTrimmed(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
|
||||
if (domainSocketPath.isEmpty()) {
|
||||
if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) &&
|
||||
(!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
|
||||
DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) {
|
||||
LOG.warn("Although short-circuit local reads are configured, " +
|
||||
"they are disabled because you didn't configure " +
|
||||
DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
if (DomainSocket.getLoadingFailureReason() != null) {
|
||||
throw new RuntimeException("Although a UNIX domain socket " +
|
||||
"path is configured as " + domainSocketPath + ", we cannot " +
|
||||
"start a localDataXceiverServer because " +
|
||||
DomainSocket.getLoadingFailureReason());
|
||||
}
|
||||
DomainPeerServer domainPeerServer =
|
||||
new DomainPeerServer(domainSocketPath, port);
|
||||
domainPeerServer.setReceiveBufferSize(
|
||||
HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
|
||||
return domainPeerServer;
|
||||
}
|
||||
|
||||
// calls specific to BP
|
||||
|
@ -1044,6 +1104,42 @@ public class DataNode extends Configured
|
|||
return info;
|
||||
}
|
||||
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
static public class ShortCircuitFdsUnsupportedException extends IOException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
public ShortCircuitFdsUnsupportedException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.LimitedPrivate("HDFS")
|
||||
static public class ShortCircuitFdsVersionException extends IOException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
public ShortCircuitFdsVersionException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
FileInputStream[] requestShortCircuitFdsForRead(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> token, int maxVersion)
|
||||
throws ShortCircuitFdsUnsupportedException,
|
||||
ShortCircuitFdsVersionException, IOException {
|
||||
if (fileDescriptorPassingDisabledReason != null) {
|
||||
throw new ShortCircuitFdsUnsupportedException(
|
||||
fileDescriptorPassingDisabledReason);
|
||||
}
|
||||
checkBlockToken(blk, token, BlockTokenSecretManager.AccessMode.READ);
|
||||
int blkVersion = CURRENT_BLOCK_FORMAT_VERSION;
|
||||
if (maxVersion < blkVersion) {
|
||||
throw new ShortCircuitFdsVersionException("Your client is too old " +
|
||||
"to read this block! Its format version is " +
|
||||
blkVersion + ", but the highest format version you can read is " +
|
||||
maxVersion);
|
||||
}
|
||||
metrics.incrBlocksGetLocalPathInfo();
|
||||
return data.getShortCircuitFdsForRead(blk);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks,
|
||||
List<Token<BlockTokenIdentifier>> tokens) throws IOException,
|
||||
|
@ -1118,32 +1214,45 @@ public class DataNode extends Configured
|
|||
if (dataXceiverServer != null) {
|
||||
((DataXceiverServer) this.dataXceiverServer.getRunnable()).kill();
|
||||
this.dataXceiverServer.interrupt();
|
||||
|
||||
// wait for all data receiver threads to exit
|
||||
if (this.threadGroup != null) {
|
||||
int sleepMs = 2;
|
||||
while (true) {
|
||||
this.threadGroup.interrupt();
|
||||
LOG.info("Waiting for threadgroup to exit, active threads is " +
|
||||
this.threadGroup.activeCount());
|
||||
if (this.threadGroup.activeCount() == 0) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(sleepMs);
|
||||
} catch (InterruptedException e) {}
|
||||
sleepMs = sleepMs * 3 / 2; // exponential backoff
|
||||
if (sleepMs > 1000) {
|
||||
sleepMs = 1000;
|
||||
}
|
||||
}
|
||||
if (localDataXceiverServer != null) {
|
||||
((DataXceiverServer) this.localDataXceiverServer.getRunnable()).kill();
|
||||
this.localDataXceiverServer.interrupt();
|
||||
}
|
||||
// wait for all data receiver threads to exit
|
||||
if (this.threadGroup != null) {
|
||||
int sleepMs = 2;
|
||||
while (true) {
|
||||
this.threadGroup.interrupt();
|
||||
LOG.info("Waiting for threadgroup to exit, active threads is " +
|
||||
this.threadGroup.activeCount());
|
||||
if (this.threadGroup.activeCount() == 0) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(sleepMs);
|
||||
} catch (InterruptedException e) {}
|
||||
sleepMs = sleepMs * 3 / 2; // exponential backoff
|
||||
if (sleepMs > 1000) {
|
||||
sleepMs = 1000;
|
||||
}
|
||||
}
|
||||
// wait for dataXceiveServer to terminate
|
||||
this.threadGroup = null;
|
||||
}
|
||||
if (this.dataXceiverServer != null) {
|
||||
// wait for dataXceiverServer to terminate
|
||||
try {
|
||||
this.dataXceiverServer.join();
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
if (this.localDataXceiverServer != null) {
|
||||
// wait for localDataXceiverServer to terminate
|
||||
try {
|
||||
this.localDataXceiverServer.join();
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
|
||||
if(blockPoolManager != null) {
|
||||
try {
|
||||
|
@ -1538,6 +1647,9 @@ public class DataNode extends Configured
|
|||
|
||||
// start dataXceiveServer
|
||||
dataXceiverServer.start();
|
||||
if (localDataXceiverServer != null) {
|
||||
localDataXceiverServer.start();
|
||||
}
|
||||
ipcServer.start();
|
||||
startPlugins(conf);
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
|
||||
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
|
||||
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
|
||||
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
|
||||
import static org.apache.hadoop.util.Time.now;
|
||||
|
@ -28,6 +29,8 @@ import java.io.BufferedOutputStream;
|
|||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.FileDescriptor;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InterruptedIOException;
|
||||
|
@ -39,6 +42,7 @@ import java.nio.channels.ClosedChannelException;
|
|||
import java.util.Arrays;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
|
@ -59,12 +63,13 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.MD5Hash;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.net.SocketInputWrapper;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
@ -79,8 +84,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
public static final Log LOG = DataNode.LOG;
|
||||
static final Log ClientTraceLog = DataNode.ClientTraceLog;
|
||||
|
||||
private final Socket s;
|
||||
private final boolean isLocal; //is a local connection?
|
||||
private final Peer peer;
|
||||
private final String remoteAddress; // address of remote side
|
||||
private final String localAddress; // local address of this daemon
|
||||
private final DataNode datanode;
|
||||
|
@ -88,7 +92,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
private final DataXceiverServer dataXceiverServer;
|
||||
private final boolean connectToDnViaHostname;
|
||||
private long opStartTime; //the start time of receiving an Op
|
||||
private final SocketInputWrapper socketIn;
|
||||
private final InputStream socketIn;
|
||||
private OutputStream socketOut;
|
||||
|
||||
/**
|
||||
|
@ -97,25 +101,23 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
*/
|
||||
private String previousOpClientName;
|
||||
|
||||
public static DataXceiver create(Socket s, DataNode dn,
|
||||
public static DataXceiver create(Peer peer, DataNode dn,
|
||||
DataXceiverServer dataXceiverServer) throws IOException {
|
||||
return new DataXceiver(s, dn, dataXceiverServer);
|
||||
return new DataXceiver(peer, dn, dataXceiverServer);
|
||||
}
|
||||
|
||||
private DataXceiver(Socket s,
|
||||
DataNode datanode,
|
||||
private DataXceiver(Peer peer, DataNode datanode,
|
||||
DataXceiverServer dataXceiverServer) throws IOException {
|
||||
|
||||
this.s = s;
|
||||
this.peer = peer;
|
||||
this.dnConf = datanode.getDnConf();
|
||||
this.socketIn = NetUtils.getInputStream(s);
|
||||
this.socketOut = NetUtils.getOutputStream(s, dnConf.socketWriteTimeout);
|
||||
this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
|
||||
this.socketIn = peer.getInputStream();
|
||||
this.socketOut = peer.getOutputStream();
|
||||
this.datanode = datanode;
|
||||
this.dataXceiverServer = dataXceiverServer;
|
||||
this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname;
|
||||
remoteAddress = s.getRemoteSocketAddress().toString();
|
||||
localAddress = s.getLocalSocketAddress().toString();
|
||||
remoteAddress = peer.getRemoteAddressString();
|
||||
localAddress = peer.getLocalAddressString();
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Number of active connections is: "
|
||||
|
@ -155,11 +157,10 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
public void run() {
|
||||
int opsProcessed = 0;
|
||||
Op op = null;
|
||||
|
||||
dataXceiverServer.childSockets.add(s);
|
||||
|
||||
|
||||
dataXceiverServer.addPeer(peer);
|
||||
try {
|
||||
|
||||
peer.setWriteTimeout(datanode.getDnConf().socketWriteTimeout);
|
||||
InputStream input = socketIn;
|
||||
if (dnConf.encryptDataTransfer) {
|
||||
IOStreamPair encryptedStreams = null;
|
||||
|
@ -169,8 +170,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
dnConf.encryptionAlgorithm);
|
||||
} catch (InvalidMagicNumberException imne) {
|
||||
LOG.info("Failed to read expected encryption handshake from client " +
|
||||
"at " + s.getInetAddress() + ". Perhaps the client is running an " +
|
||||
"older version of Hadoop which does not support encryption");
|
||||
"at " + peer.getRemoteAddressString() + ". Perhaps the client " +
|
||||
"is running an older version of Hadoop which does not support " +
|
||||
"encryption");
|
||||
return;
|
||||
}
|
||||
input = encryptedStreams.in;
|
||||
|
@ -189,9 +191,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
try {
|
||||
if (opsProcessed != 0) {
|
||||
assert dnConf.socketKeepaliveTimeout > 0;
|
||||
socketIn.setTimeout(dnConf.socketKeepaliveTimeout);
|
||||
peer.setReadTimeout(dnConf.socketKeepaliveTimeout);
|
||||
} else {
|
||||
socketIn.setTimeout(dnConf.socketTimeout);
|
||||
peer.setReadTimeout(dnConf.socketTimeout);
|
||||
}
|
||||
op = readOp();
|
||||
} catch (InterruptedIOException ignored) {
|
||||
|
@ -202,7 +204,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
if (opsProcessed > 0 &&
|
||||
(err instanceof EOFException || err instanceof ClosedChannelException)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Cached " + s.toString() + " closing after " + opsProcessed + " ops");
|
||||
LOG.debug("Cached " + peer + " closing after " + opsProcessed + " ops");
|
||||
}
|
||||
} else {
|
||||
throw err;
|
||||
|
@ -212,13 +214,13 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
|
||||
// restore normal timeout
|
||||
if (opsProcessed != 0) {
|
||||
s.setSoTimeout(dnConf.socketTimeout);
|
||||
peer.setReadTimeout(dnConf.socketTimeout);
|
||||
}
|
||||
|
||||
opStartTime = now();
|
||||
processOp(op);
|
||||
++opsProcessed;
|
||||
} while (!s.isClosed() && dnConf.socketKeepaliveTimeout > 0);
|
||||
} while (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0);
|
||||
} catch (Throwable t) {
|
||||
LOG.error(datanode.getDisplayName() + ":DataXceiver error processing " +
|
||||
((op == null) ? "unknown" : op.name()) + " operation " +
|
||||
|
@ -230,9 +232,70 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
+ datanode.getXceiverCount());
|
||||
}
|
||||
updateCurrentThreadName("Cleaning up");
|
||||
dataXceiverServer.closePeer(peer);
|
||||
IOUtils.closeStream(in);
|
||||
IOUtils.closeSocket(s);
|
||||
dataXceiverServer.childSockets.remove(s);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> token,
|
||||
int maxVersion) throws IOException {
|
||||
updateCurrentThreadName("Passing file descriptors for block " + blk);
|
||||
BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
|
||||
FileInputStream fis[] = null;
|
||||
try {
|
||||
if (peer.getDomainSocket() == null) {
|
||||
throw new IOException("You cannot pass file descriptors over " +
|
||||
"anything but a UNIX domain socket.");
|
||||
}
|
||||
fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
|
||||
bld.setStatus(SUCCESS);
|
||||
bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
|
||||
} catch (ShortCircuitFdsVersionException e) {
|
||||
bld.setStatus(ERROR_UNSUPPORTED);
|
||||
bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
|
||||
bld.setMessage(e.getMessage());
|
||||
} catch (ShortCircuitFdsUnsupportedException e) {
|
||||
bld.setStatus(ERROR_UNSUPPORTED);
|
||||
bld.setMessage(e.getMessage());
|
||||
} catch (InvalidToken e) {
|
||||
bld.setStatus(ERROR_ACCESS_TOKEN);
|
||||
bld.setMessage(e.getMessage());
|
||||
} catch (IOException e) {
|
||||
bld.setStatus(ERROR);
|
||||
bld.setMessage(e.getMessage());
|
||||
}
|
||||
try {
|
||||
bld.build().writeDelimitedTo(socketOut);
|
||||
if (fis != null) {
|
||||
FileDescriptor fds[] = new FileDescriptor[fis.length];
|
||||
for (int i = 0; i < fds.length; i++) {
|
||||
fds[i] = fis[i].getFD();
|
||||
}
|
||||
byte buf[] = new byte[] { (byte)0 };
|
||||
peer.getDomainSocket().
|
||||
sendFileDescriptors(fds, buf, 0, buf.length);
|
||||
}
|
||||
} finally {
|
||||
if (ClientTraceLog.isInfoEnabled()) {
|
||||
DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
|
||||
.getBlockPoolId());
|
||||
BlockSender.ClientTraceLog.info(String.format(
|
||||
String.format(
|
||||
"src: %s, dest: %s, op: %s, blockid: %s, srvID: %s, " +
|
||||
"success: %b",
|
||||
"127.0.0.1", // src IP
|
||||
"127.0.0.1", // dst IP
|
||||
"REQUEST_SHORT_CIRCUIT_FDS", // operation
|
||||
blk.getBlockId(), // block id
|
||||
dnR.getStorageID(),
|
||||
(fis != null)
|
||||
)));
|
||||
}
|
||||
if (fis != null) {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -287,8 +350,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
|
||||
PBHelper.vintPrefixed(in));
|
||||
if (!stat.hasStatus()) {
|
||||
LOG.warn("Client " + s.getInetAddress() + " did not send a valid status " +
|
||||
"code after reading. Will close connection.");
|
||||
LOG.warn("Client " + peer.getRemoteAddressString() +
|
||||
" did not send a valid status code after reading. " +
|
||||
"Will close connection.");
|
||||
IOUtils.closeStream(out);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
|
@ -321,7 +385,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
|
||||
//update metrics
|
||||
datanode.metrics.addReadBlockOp(elapsed());
|
||||
datanode.metrics.incrReadsFromClient(isLocal);
|
||||
datanode.metrics.incrReadsFromClient(peer.isLocal());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -359,8 +423,8 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
LOG.debug("isDatanode=" + isDatanode
|
||||
+ ", isClient=" + isClient
|
||||
+ ", isTransfer=" + isTransfer);
|
||||
LOG.debug("writeBlock receive buf size " + s.getReceiveBufferSize() +
|
||||
" tcp no delay " + s.getTcpNoDelay());
|
||||
LOG.debug("writeBlock receive buf size " + peer.getReceiveBufferSize() +
|
||||
" tcp no delay " + peer.getTcpNoDelay());
|
||||
}
|
||||
|
||||
// We later mutate block's generation stamp and length, but we need to
|
||||
|
@ -391,8 +455,8 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
|
||||
// open a block receiver
|
||||
blockReceiver = new BlockReceiver(block, in,
|
||||
s.getRemoteSocketAddress().toString(),
|
||||
s.getLocalSocketAddress().toString(),
|
||||
peer.getRemoteAddressString(),
|
||||
peer.getLocalAddressString(),
|
||||
stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
|
||||
clientname, srcDataNode, datanode, requestedChecksum);
|
||||
} else {
|
||||
|
@ -547,7 +611,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
|
||||
//update metrics
|
||||
datanode.metrics.addWriteBlockOp(elapsed());
|
||||
datanode.metrics.incrWritesFromClient(isLocal);
|
||||
datanode.metrics.incrWritesFromClient(peer.isLocal());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -555,7 +619,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
final Token<BlockTokenIdentifier> blockToken,
|
||||
final String clientName,
|
||||
final DatanodeInfo[] targets) throws IOException {
|
||||
checkAccess(null, true, blk, blockToken,
|
||||
checkAccess(socketOut, true, blk, blockToken,
|
||||
Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
|
||||
previousOpClientName = clientName;
|
||||
updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
|
||||
|
@ -642,8 +706,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
}
|
||||
|
||||
if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
|
||||
String msg = "Not able to copy block " + block.getBlockId() + " to "
|
||||
+ s.getRemoteSocketAddress() + " because threads quota is exceeded.";
|
||||
String msg = "Not able to copy block " + block.getBlockId() + " " +
|
||||
"to " + peer.getRemoteAddressString() + " because threads " +
|
||||
"quota is exceeded.";
|
||||
LOG.info(msg);
|
||||
sendResponse(ERROR, msg);
|
||||
return;
|
||||
|
@ -672,7 +737,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
datanode.metrics.incrBytesRead((int) read);
|
||||
datanode.metrics.incrBlocksRead();
|
||||
|
||||
LOG.info("Copied " + block + " to " + s.getRemoteSocketAddress());
|
||||
LOG.info("Copied " + block + " to " + peer.getRemoteAddressString());
|
||||
} catch (IOException ioe) {
|
||||
isOpSuccess = false;
|
||||
LOG.info("opCopyBlock " + block + " received exception " + ioe);
|
||||
|
@ -717,8 +782,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
}
|
||||
|
||||
if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
|
||||
String msg = "Not able to receive block " + block.getBlockId() + " from "
|
||||
+ s.getRemoteSocketAddress() + " because threads quota is exceeded.";
|
||||
String msg = "Not able to receive block " + block.getBlockId() +
|
||||
" from " + peer.getRemoteAddressString() + " because threads " +
|
||||
"quota is exceeded.";
|
||||
LOG.warn(msg);
|
||||
sendResponse(ERROR, msg);
|
||||
return;
|
||||
|
@ -795,7 +861,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
// notify name node
|
||||
datanode.notifyNamenodeReceivedBlock(block, delHint);
|
||||
|
||||
LOG.info("Moved " + block + " from " + s.getRemoteSocketAddress());
|
||||
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString());
|
||||
|
||||
} catch (IOException ioe) {
|
||||
opStatus = ERROR;
|
||||
|
@ -818,7 +884,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
try {
|
||||
sendResponse(opStatus, errMsg);
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Error writing reply back to " + s.getRemoteSocketAddress());
|
||||
LOG.warn("Error writing reply back to " + peer.getRemoteAddressString());
|
||||
}
|
||||
IOUtils.closeStream(proxyOut);
|
||||
IOUtils.closeStream(blockReceiver);
|
||||
|
@ -872,7 +938,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
}
|
||||
|
||||
|
||||
private void checkAccess(DataOutputStream out, final boolean reply,
|
||||
private void checkAccess(OutputStream out, final boolean reply,
|
||||
final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> t,
|
||||
final Op op,
|
||||
|
@ -887,11 +953,6 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
} catch(InvalidToken e) {
|
||||
try {
|
||||
if (reply) {
|
||||
if (out == null) {
|
||||
out = new DataOutputStream(
|
||||
NetUtils.getOutputStream(s, dnConf.socketWriteTimeout));
|
||||
}
|
||||
|
||||
BlockOpResponseProto.Builder resp = BlockOpResponseProto.newBuilder()
|
||||
.setStatus(ERROR_ACCESS_TOKEN);
|
||||
if (mode == BlockTokenSecretManager.AccessMode.WRITE) {
|
||||
|
|
|
@ -18,18 +18,16 @@
|
|||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.channels.AsynchronousCloseException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.PeerServer;
|
||||
import org.apache.hadoop.hdfs.server.balancer.Balancer;
|
||||
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
@ -45,11 +43,9 @@ import org.apache.hadoop.util.Daemon;
|
|||
class DataXceiverServer implements Runnable {
|
||||
public static final Log LOG = DataNode.LOG;
|
||||
|
||||
ServerSocket ss;
|
||||
DataNode datanode;
|
||||
// Record all sockets opened for data transfer
|
||||
Set<Socket> childSockets = Collections.synchronizedSet(
|
||||
new HashSet<Socket>());
|
||||
private final PeerServer peerServer;
|
||||
private final DataNode datanode;
|
||||
private final Set<Peer> peers = new HashSet<Peer>();
|
||||
|
||||
/**
|
||||
* Maximal number of concurrent xceivers per node.
|
||||
|
@ -109,10 +105,10 @@ class DataXceiverServer implements Runnable {
|
|||
long estimateBlockSize;
|
||||
|
||||
|
||||
DataXceiverServer(ServerSocket ss, Configuration conf,
|
||||
DataXceiverServer(PeerServer peerServer, Configuration conf,
|
||||
DataNode datanode) {
|
||||
|
||||
this.ss = ss;
|
||||
this.peerServer = peerServer;
|
||||
this.datanode = datanode;
|
||||
|
||||
this.maxXceiverCount =
|
||||
|
@ -130,12 +126,10 @@ class DataXceiverServer implements Runnable {
|
|||
|
||||
@Override
|
||||
public void run() {
|
||||
Peer peer = null;
|
||||
while (datanode.shouldRun) {
|
||||
Socket s = null;
|
||||
try {
|
||||
s = ss.accept();
|
||||
s.setTcpNoDelay(true);
|
||||
// Timeouts are set within DataXceiver.run()
|
||||
peer = peerServer.accept();
|
||||
|
||||
// Make sure the xceiver count is not exceeded
|
||||
int curXceiverCount = datanode.getXceiverCount();
|
||||
|
@ -146,7 +140,7 @@ class DataXceiverServer implements Runnable {
|
|||
}
|
||||
|
||||
new Daemon(datanode.threadGroup,
|
||||
DataXceiver.create(s, datanode, this))
|
||||
DataXceiver.create(peer, datanode, this))
|
||||
.start();
|
||||
} catch (SocketTimeoutException ignored) {
|
||||
// wake up to see if should continue to run
|
||||
|
@ -157,10 +151,10 @@ class DataXceiverServer implements Runnable {
|
|||
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ace);
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
IOUtils.closeSocket(s);
|
||||
IOUtils.cleanup(null, peer);
|
||||
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ie);
|
||||
} catch (OutOfMemoryError ie) {
|
||||
IOUtils.closeSocket(s);
|
||||
IOUtils.cleanup(null, peer);
|
||||
// DataNode can run out of memory if there is too many transfers.
|
||||
// Log the event, Sleep for 30 seconds, other transfers may complete by
|
||||
// then.
|
||||
|
@ -176,33 +170,35 @@ class DataXceiverServer implements Runnable {
|
|||
datanode.shouldRun = false;
|
||||
}
|
||||
}
|
||||
synchronized (this) {
|
||||
for (Peer p : peers) {
|
||||
IOUtils.cleanup(LOG, p);
|
||||
}
|
||||
}
|
||||
try {
|
||||
ss.close();
|
||||
peerServer.close();
|
||||
} catch (IOException ie) {
|
||||
LOG.warn(datanode.getDisplayName()
|
||||
+ " :DataXceiverServer: close exception", ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void kill() {
|
||||
assert datanode.shouldRun == false :
|
||||
"shoudRun should be set to false before killing";
|
||||
try {
|
||||
this.ss.close();
|
||||
this.peerServer.close();
|
||||
} catch (IOException ie) {
|
||||
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer.kill(): ", ie);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void addPeer(Peer peer) {
|
||||
peers.add(peer);
|
||||
}
|
||||
|
||||
// close all the sockets that were accepted earlier
|
||||
synchronized (childSockets) {
|
||||
for (Iterator<Socket> it = childSockets.iterator();
|
||||
it.hasNext();) {
|
||||
Socket thissock = it.next();
|
||||
try {
|
||||
thissock.close();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
synchronized void closePeer(Peer peer) {
|
||||
peers.remove(peer);
|
||||
IOUtils.cleanup(null, peer);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
|
|||
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
|
@ -386,4 +387,6 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
|
|||
public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
|
||||
throws IOException;
|
||||
|
||||
FileInputStream[] getShortCircuitFdsForRead(ExtendedBlock block)
|
||||
throws IOException;
|
||||
}
|
||||
|
|
|
@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
|
|||
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
||||
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.metrics2.util.MBeans;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
@ -1700,6 +1701,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
return info;
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public FileInputStream[] getShortCircuitFdsForRead(ExtendedBlock block)
|
||||
throws IOException {
|
||||
File datafile = getBlockFile(block);
|
||||
File metafile = FsDatasetUtil.getMetaFile(datafile,
|
||||
block.getGenerationStamp());
|
||||
FileInputStream fis[] = new FileInputStream[2];
|
||||
boolean success = false;
|
||||
try {
|
||||
fis[0] = new FileInputStream(datafile);
|
||||
fis[1] = new FileInputStream(metafile);
|
||||
success = true;
|
||||
return fis;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.cleanup(null, fis);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
|
||||
throws IOException {
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.BlockReaderFactory;
|
|||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -559,9 +560,10 @@ public class NamenodeFsck {
|
|||
String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(),
|
||||
block.getBlockId());
|
||||
blockReader = BlockReaderFactory.newBlockReader(
|
||||
conf, s, file, block, lblock
|
||||
.getBlockToken(), 0, -1,
|
||||
namenode.getRpcServer().getDataEncryptionKey());
|
||||
conf, file, block, lblock.getBlockToken(), 0, -1, true, "fsck",
|
||||
TcpPeerServer.peerFromSocketAndKey(s, namenode.getRpcServer().
|
||||
getDataEncryptionKey()),
|
||||
chosenNode, null, false);
|
||||
|
||||
} catch (IOException ex) {
|
||||
// Put chosen node into dead list, continue
|
||||
|
|
|
@ -74,6 +74,8 @@ message DeleteBlockPoolResponseProto {
|
|||
* Gets the file information where block and its metadata is stored
|
||||
* block - block for which path information is being requested
|
||||
* token - block token
|
||||
*
|
||||
* This message is deprecated in favor of file descriptor passing.
|
||||
*/
|
||||
message GetBlockLocalPathInfoRequestProto {
|
||||
required ExtendedBlockProto block = 1;
|
||||
|
@ -84,6 +86,8 @@ message GetBlockLocalPathInfoRequestProto {
|
|||
* block - block for which file path information is being returned
|
||||
* localPath - file path where the block data is stored
|
||||
* localMetaPath - file path where the block meta data is stored
|
||||
*
|
||||
* This message is deprecated in favor of file descriptor passing.
|
||||
*/
|
||||
message GetBlockLocalPathInfoResponseProto {
|
||||
required ExtendedBlockProto block = 1;
|
||||
|
|
|
@ -115,6 +115,16 @@ message OpBlockChecksumProto {
|
|||
required BaseHeaderProto header = 1;
|
||||
}
|
||||
|
||||
message OpRequestShortCircuitAccessProto {
|
||||
required BaseHeaderProto header = 1;
|
||||
|
||||
/** In order to get short-circuit access to block data, clients must set this
|
||||
* to the highest version of the block data that they can understand.
|
||||
* Currently 1 is the only version, but more versions may exist in the future
|
||||
* if the on-disk format changes.
|
||||
*/
|
||||
required uint32 maxVersion = 2;
|
||||
}
|
||||
|
||||
message PacketHeaderProto {
|
||||
// All fields must be fixed-length!
|
||||
|
@ -133,6 +143,7 @@ enum Status {
|
|||
ERROR_EXISTS = 4;
|
||||
ERROR_ACCESS_TOKEN = 5;
|
||||
CHECKSUM_OK = 6;
|
||||
ERROR_UNSUPPORTED = 7;
|
||||
}
|
||||
|
||||
message PipelineAckProto {
|
||||
|
@ -165,6 +176,16 @@ message BlockOpResponseProto {
|
|||
|
||||
/** explanatory text which may be useful to log on the client side */
|
||||
optional string message = 5;
|
||||
|
||||
/** If the server chooses to agree to the request of a client for
|
||||
* short-circuit access, it will send a response message with the relevant
|
||||
* file descriptors attached.
|
||||
*
|
||||
* In the body of the message, this version number will be set to the
|
||||
* specific version number of the block data that the client is about to
|
||||
* read.
|
||||
*/
|
||||
optional uint32 shortCircuitAccessVersion = 6;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1231,6 +1231,17 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.domain.socket.path</name>
|
||||
<value></value>
|
||||
<description>
|
||||
Optional. This is a path to a UNIX domain socket that will be used for
|
||||
communication between the DataNode and local HDFS clients.
|
||||
If the string "_PORT" is present in this path, it will be replaced by the
|
||||
TCP port of the DataNode.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.fsdataset.volume.choosing.balanced-space-threshold</name>
|
||||
<value>10737418240</value> <!-- 10 GB -->
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
|
||||
~~ Licensed 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. See accompanying LICENSE file.
|
||||
|
||||
---
|
||||
Hadoop Distributed File System-${project.version} - Short-Circuit Local Reads
|
||||
---
|
||||
---
|
||||
${maven.build.timestamp}
|
||||
|
||||
HDFS Short-Circuit Local Reads
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=0}
|
||||
|
||||
* {Background}
|
||||
|
||||
In <<<HDFS>>>, reads normally go through the <<<DataNode>>>. Thus, when the
|
||||
client asks the <<<DataNode>>> to read a file, the <<<DataNode>>> reads that
|
||||
file off of the disk and sends the data to the client over a TCP socket.
|
||||
So-called "short-circuit" reads bypass the <<<DataNode>>>, allowing the client
|
||||
to read the file directly. Obviously, this is only possible in cases where
|
||||
the client is co-located with the data. Short-circuit reads provide a
|
||||
substantial performance boost to many applications.
|
||||
|
||||
* {Configuration}
|
||||
|
||||
To configure short-circuit local reads, you will need to enable
|
||||
<<<libhadoop.so>>>. See
|
||||
{{{../hadoop-common/NativeLibraries.html}Native
|
||||
Libraries}} for details on enabling this library.
|
||||
|
||||
Short-circuit reads make use of a UNIX domain socket. This is a special path
|
||||
in the filesystem that allows the client and the DataNodes to communicate.
|
||||
You will need to set a path to this socket. The DataNode needs to be able to
|
||||
create this path. On the other hand, it should not be possible for any user
|
||||
except the hdfs user or root to create this path. For this reason, paths
|
||||
under <<</var/run>>> or <<</var/lib>>> are often used.
|
||||
|
||||
Short-circuit local reads need to be configured on both the <<<DataNode>>>
|
||||
and the client.
|
||||
|
||||
* {Example Configuration}
|
||||
|
||||
Here is an example configuration.
|
||||
|
||||
----
|
||||
<configuration>
|
||||
<property>
|
||||
<name>dfs.client.read.shortcircuit</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.domain.socket.path</name>
|
||||
<value>/var/lib/hadoop-hdfs/dn_socket</value>
|
||||
</property>
|
||||
</configuration>
|
||||
----
|
|
@ -28,9 +28,9 @@ import java.net.Socket;
|
|||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
|
@ -150,12 +150,12 @@ public class BlockReaderTestUtil {
|
|||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
|
||||
return BlockReaderFactory.newBlockReader(
|
||||
new DFSClient.Conf(conf),
|
||||
sock, targetAddr.toString()+ ":" + block.getBlockId(), block,
|
||||
conf,
|
||||
targetAddr.toString()+ ":" + block.getBlockId(), block,
|
||||
testBlock.getBlockToken(),
|
||||
offset, lenToRead,
|
||||
conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
|
||||
true, "", null, null);
|
||||
true, "BlockReaderTestUtil", TcpPeerServer.peerFromSocket(sock),
|
||||
nodes[0], null, false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -166,5 +166,4 @@ public class BlockReaderTestUtil {
|
|||
int ipcport = nodes[0].getIpcPort();
|
||||
return cluster.getDataNode(ipcport);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -2188,14 +2188,27 @@ public class MiniDFSCluster {
|
|||
/**
|
||||
* Get file correpsonding to a block
|
||||
* @param storageDir storage directory
|
||||
* @param blk block to be corrupted
|
||||
* @return file corresponding to the block
|
||||
* @param blk the block
|
||||
* @return data file corresponding to the block
|
||||
*/
|
||||
public static File getBlockFile(File storageDir, ExtendedBlock blk) {
|
||||
return new File(getFinalizedDir(storageDir, blk.getBlockPoolId()),
|
||||
blk.getBlockName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest metadata file correpsonding to a block
|
||||
* @param storageDir storage directory
|
||||
* @param blk the block
|
||||
* @return metadata file corresponding to the block
|
||||
*/
|
||||
public static File getBlockMetadataFile(File storageDir, ExtendedBlock blk) {
|
||||
return new File(getFinalizedDir(storageDir, blk.getBlockPoolId()),
|
||||
blk.getBlockName() + "_" + blk.getGenerationStamp() +
|
||||
Block.METADATA_EXTENSION);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Shut down a cluster if it is not null
|
||||
* @param cluster cluster reference or null
|
||||
|
@ -2223,7 +2236,7 @@ public class MiniDFSCluster {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get files related to a block for a given datanode
|
||||
* Get the block data file for a block from a given datanode
|
||||
* @param dnIndex Index of the datanode to get block files for
|
||||
* @param block block for which corresponding files are needed
|
||||
*/
|
||||
|
@ -2238,6 +2251,24 @@ public class MiniDFSCluster {
|
|||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the block metadata file for a block from a given datanode
|
||||
*
|
||||
* @param dnIndex Index of the datanode to get block files for
|
||||
* @param block block for which corresponding files are needed
|
||||
*/
|
||||
public static File getBlockMetadataFile(int dnIndex, ExtendedBlock block) {
|
||||
// Check for block file in the two storage directories of the datanode
|
||||
for (int i = 0; i <=1 ; i++) {
|
||||
File storageDir = MiniDFSCluster.getStorageDir(dnIndex, i);
|
||||
File blockMetaFile = getBlockMetadataFile(storageDir, block);
|
||||
if (blockMetaFile.exists()) {
|
||||
return blockMetaFile;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Throw an exception if the MiniDFSCluster is not started with a single
|
||||
|
|
|
@ -17,90 +17,333 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
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.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestBlockReaderLocal {
|
||||
static MiniDFSCluster cluster;
|
||||
static HdfsConfiguration conf;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws IOException {
|
||||
conf = new HdfsConfiguration();
|
||||
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
false);
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
UserGroupInformation.getCurrentUser().getShortUserName());
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownCluster() {
|
||||
cluster.shutdown();
|
||||
public static void assertArrayRegionsEqual(byte []buf1, int off1, byte []buf2,
|
||||
int off2, int len) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (buf1[off1 + i] != buf2[off2 + i]) {
|
||||
Assert.fail("arrays differ at byte " + i + ". " +
|
||||
"The first array has " + (int)buf1[off1 + i] +
|
||||
", but the second array has " + (int)buf2[off2 + i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that, in the case of an error, the position and limit of a ByteBuffer
|
||||
* are left unchanged. This is not mandated by ByteBufferReadable, but clients
|
||||
* of this class might immediately issue a retry on failure, so it's polite.
|
||||
* Similar to IOUtils#readFully(). Reads bytes in a loop.
|
||||
*
|
||||
* @param reader The BlockReaderLocal to read bytes from
|
||||
* @param buf The ByteBuffer to read into
|
||||
* @param off The offset in the buffer to read into
|
||||
* @param len The number of bytes to read.
|
||||
*
|
||||
* @throws IOException If it could not read the requested number of bytes
|
||||
*/
|
||||
private static void readFully(BlockReaderLocal reader,
|
||||
ByteBuffer buf, int off, int len) throws IOException {
|
||||
int amt = len;
|
||||
while (amt > 0) {
|
||||
buf.limit(off + len);
|
||||
buf.position(off);
|
||||
long ret = reader.read(buf);
|
||||
if (ret < 0) {
|
||||
throw new EOFException( "Premature EOF from BlockReaderLocal " +
|
||||
"after reading " + (len - amt) + " byte(s).");
|
||||
}
|
||||
amt -= ret;
|
||||
off += ret;
|
||||
}
|
||||
}
|
||||
|
||||
private static interface BlockReaderLocalTest {
|
||||
final int TEST_LENGTH = 12345;
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException;
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException;
|
||||
}
|
||||
|
||||
public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
||||
boolean checksum) throws IOException {
|
||||
MiniDFSCluster cluster = null;
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, !checksum);
|
||||
conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
|
||||
FileInputStream dataIn = null, checkIn = null;
|
||||
final Path TEST_PATH = new Path("/a");
|
||||
final long RANDOM_SEED = 4567L;
|
||||
BlockReaderLocal blockReaderLocal = null;
|
||||
FSDataInputStream fsIn = null;
|
||||
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
||||
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||
BlockReaderLocalTest.TEST_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);
|
||||
IOUtils.readFully(fsIn, original, 0,
|
||||
BlockReaderLocalTest.TEST_LENGTH);
|
||||
fsIn.close();
|
||||
fsIn = null;
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
||||
File dataFile = MiniDFSCluster.getBlockFile(0, block);
|
||||
File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block);
|
||||
|
||||
DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
test.setup(dataFile, checksum);
|
||||
dataIn = new FileInputStream(dataFile);
|
||||
checkIn = new FileInputStream(metaFile);
|
||||
blockReaderLocal = new BlockReaderLocal(conf,
|
||||
TEST_PATH.getName(), block, 0, -1,
|
||||
dataIn, checkIn, datanodeID, checksum);
|
||||
dataIn = null;
|
||||
checkIn = null;
|
||||
test.doTest(blockReaderLocal, original);
|
||||
} finally {
|
||||
if (fsIn != null) fsIn.close();
|
||||
if (cluster != null) cluster.shutdown();
|
||||
if (dataIn != null) dataIn.close();
|
||||
if (checkIn != null) checkIn.close();
|
||||
if (blockReaderLocal != null) blockReaderLocal.close(null, null);
|
||||
}
|
||||
}
|
||||
|
||||
private static class TestBlockReaderLocalImmediateClose
|
||||
implements BlockReaderLocalTest {
|
||||
@Override
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException { }
|
||||
@Override
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException { }
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStablePositionAfterCorruptRead() throws Exception {
|
||||
final short REPL_FACTOR = 1;
|
||||
final long FILE_LENGTH = 512L;
|
||||
cluster.waitActive();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
public void testBlockReaderLocalImmediateClose() throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true);
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false);
|
||||
}
|
||||
|
||||
private static class TestBlockReaderSimpleReads
|
||||
implements BlockReaderLocalTest {
|
||||
@Override
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException { }
|
||||
@Override
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException {
|
||||
byte buf[] = new byte[TEST_LENGTH];
|
||||
reader.readFully(buf, 0, 512);
|
||||
assertArrayRegionsEqual(original, 0, buf, 0, 512);
|
||||
reader.readFully(buf, 512, 512);
|
||||
assertArrayRegionsEqual(original, 512, buf, 512, 512);
|
||||
reader.readFully(buf, 1024, 513);
|
||||
assertArrayRegionsEqual(original, 1024, buf, 1024, 513);
|
||||
reader.readFully(buf, 1537, 514);
|
||||
assertArrayRegionsEqual(original, 1537, buf, 1537, 514);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockReaderSimpleReads() throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true);
|
||||
}
|
||||
|
||||
Path path = new Path("/corrupted");
|
||||
@Test
|
||||
public void testBlockReaderSimpleReadsNoChecksum() throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false);
|
||||
}
|
||||
|
||||
private static class TestBlockReaderLocalArrayReads2
|
||||
implements BlockReaderLocalTest {
|
||||
@Override
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException { }
|
||||
@Override
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException {
|
||||
byte buf[] = new byte[TEST_LENGTH];
|
||||
reader.readFully(buf, 0, 10);
|
||||
assertArrayRegionsEqual(original, 0, buf, 0, 10);
|
||||
reader.readFully(buf, 10, 100);
|
||||
assertArrayRegionsEqual(original, 10, buf, 10, 100);
|
||||
reader.readFully(buf, 110, 700);
|
||||
assertArrayRegionsEqual(original, 110, buf, 110, 700);
|
||||
reader.readFully(buf, 810, 1); // from offset 810 to offset 811
|
||||
reader.readFully(buf, 811, 5);
|
||||
assertArrayRegionsEqual(original, 811, buf, 811, 5);
|
||||
reader.readFully(buf, 816, 900); // skip from offset 816 to offset 1716
|
||||
reader.readFully(buf, 1716, 5);
|
||||
assertArrayRegionsEqual(original, 1716, buf, 1716, 5);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockReaderLocalArrayReads2() throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
|
||||
true);
|
||||
}
|
||||
|
||||
DFSTestUtil.createFile(fs, path, FILE_LENGTH, REPL_FACTOR, 12345L);
|
||||
DFSTestUtil.waitReplication(fs, path, REPL_FACTOR);
|
||||
@Test
|
||||
public void testBlockReaderLocalArrayReads2NoChecksum()
|
||||
throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
|
||||
false);
|
||||
}
|
||||
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, path);
|
||||
int blockFilesCorrupted = cluster.corruptBlockOnDataNodes(block);
|
||||
assertEquals("All replicas not corrupted", REPL_FACTOR, blockFilesCorrupted);
|
||||
private static class TestBlockReaderLocalByteBufferReads
|
||||
implements BlockReaderLocalTest {
|
||||
@Override
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException { }
|
||||
@Override
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException {
|
||||
ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
|
||||
readFully(reader, buf, 0, 10);
|
||||
assertArrayRegionsEqual(original, 0, buf.array(), 0, 10);
|
||||
readFully(reader, buf, 10, 100);
|
||||
assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
|
||||
readFully(reader, buf, 110, 700);
|
||||
assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
|
||||
reader.skip(1); // skip from offset 810 to offset 811
|
||||
readFully(reader, buf, 811, 5);
|
||||
assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockReaderLocalByteBufferReads()
|
||||
throws IOException {
|
||||
runBlockReaderLocalTest(
|
||||
new TestBlockReaderLocalByteBufferReads(), true);
|
||||
}
|
||||
|
||||
FSDataInputStream dis = cluster.getFileSystem().open(path);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect((int)FILE_LENGTH);
|
||||
boolean sawException = false;
|
||||
try {
|
||||
dis.read(buf);
|
||||
} catch (ChecksumException ex) {
|
||||
sawException = true;
|
||||
@Test
|
||||
public void testBlockReaderLocalByteBufferReadsNoChecksum()
|
||||
throws IOException {
|
||||
runBlockReaderLocalTest(
|
||||
new TestBlockReaderLocalByteBufferReads(), false);
|
||||
}
|
||||
|
||||
private static class TestBlockReaderLocalReadCorruptStart
|
||||
implements BlockReaderLocalTest {
|
||||
boolean usingChecksums = false;
|
||||
@Override
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException {
|
||||
RandomAccessFile bf = null;
|
||||
this.usingChecksums = usingChecksums;
|
||||
try {
|
||||
bf = new RandomAccessFile(blockFile, "rw");
|
||||
bf.write(new byte[] {0,0,0,0,0,0,0,0,0,0,0,0,0,0});
|
||||
} finally {
|
||||
if (bf != null) bf.close();
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(sawException);
|
||||
assertEquals(0, buf.position());
|
||||
assertEquals(buf.capacity(), buf.limit());
|
||||
|
||||
dis = cluster.getFileSystem().open(path);
|
||||
buf.position(3);
|
||||
buf.limit(25);
|
||||
sawException = false;
|
||||
try {
|
||||
dis.read(buf);
|
||||
} catch (ChecksumException ex) {
|
||||
sawException = true;
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException {
|
||||
byte buf[] = new byte[TEST_LENGTH];
|
||||
if (usingChecksums) {
|
||||
try {
|
||||
reader.readFully(buf, 0, 10);
|
||||
Assert.fail("did not detect corruption");
|
||||
} catch (IOException e) {
|
||||
// expected
|
||||
}
|
||||
} else {
|
||||
reader.readFully(buf, 0, 10);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockReaderLocalReadCorruptStart()
|
||||
throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true);
|
||||
}
|
||||
|
||||
private static class TestBlockReaderLocalReadCorrupt
|
||||
implements BlockReaderLocalTest {
|
||||
boolean usingChecksums = false;
|
||||
@Override
|
||||
public void setup(File blockFile, boolean usingChecksums)
|
||||
throws IOException {
|
||||
RandomAccessFile bf = null;
|
||||
this.usingChecksums = usingChecksums;
|
||||
try {
|
||||
bf = new RandomAccessFile(blockFile, "rw");
|
||||
bf.seek(1539);
|
||||
bf.write(new byte[] {0,0,0,0,0,0,0,0,0,0,0,0,0,0});
|
||||
} finally {
|
||||
if (bf != null) bf.close();
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(sawException);
|
||||
assertEquals(3, buf.position());
|
||||
assertEquals(25, buf.limit());
|
||||
public void doTest(BlockReaderLocal reader, byte original[])
|
||||
throws IOException {
|
||||
byte buf[] = new byte[TEST_LENGTH];
|
||||
try {
|
||||
reader.readFully(buf, 0, 10);
|
||||
assertArrayRegionsEqual(original, 0, buf, 0, 10);
|
||||
reader.readFully(buf, 10, 100);
|
||||
assertArrayRegionsEqual(original, 10, buf, 10, 100);
|
||||
reader.readFully(buf, 110, 700);
|
||||
assertArrayRegionsEqual(original, 110, buf, 110, 700);
|
||||
reader.skip(1); // skip from offset 810 to offset 811
|
||||
reader.readFully(buf, 811, 5);
|
||||
assertArrayRegionsEqual(original, 811, buf, 811, 5);
|
||||
reader.readFully(buf, 816, 900);
|
||||
if (usingChecksums) {
|
||||
// We should detect the corruption when using a checksum file.
|
||||
Assert.fail("did not detect corruption");
|
||||
}
|
||||
} catch (ChecksumException e) {
|
||||
if (!usingChecksums) {
|
||||
Assert.fail("didn't expect to get ChecksumException: not " +
|
||||
"using checksums.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockReaderLocalReadCorrupt()
|
||||
throws IOException {
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true);
|
||||
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,154 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.security.UserGroupInformation;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestBlockReaderLocalLegacy {
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws IOException {
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
DomainSocket.disableBindPathValidation();
|
||||
}
|
||||
|
||||
private static HdfsConfiguration getConfiguration(
|
||||
TemporarySocketDirectory socketDir) throws IOException {
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
if (socketDir == null) {
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "");
|
||||
} else {
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(socketDir.getDir(), "TestBlockReaderLocalLegacy.%d.sock").
|
||||
getAbsolutePath());
|
||||
}
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
false);
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
UserGroupInformation.getCurrentUser().getShortUserName());
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
|
||||
return conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that, in the case of an error, the position and limit of a ByteBuffer
|
||||
* are left unchanged. This is not mandated by ByteBufferReadable, but clients
|
||||
* of this class might immediately issue a retry on failure, so it's polite.
|
||||
*/
|
||||
@Test
|
||||
public void testStablePositionAfterCorruptRead() throws Exception {
|
||||
final short REPL_FACTOR = 1;
|
||||
final long FILE_LENGTH = 512L;
|
||||
|
||||
HdfsConfiguration conf = getConfiguration(null);
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
|
||||
Path path = new Path("/corrupted");
|
||||
|
||||
DFSTestUtil.createFile(fs, path, FILE_LENGTH, REPL_FACTOR, 12345L);
|
||||
DFSTestUtil.waitReplication(fs, path, REPL_FACTOR);
|
||||
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, path);
|
||||
int blockFilesCorrupted = cluster.corruptBlockOnDataNodes(block);
|
||||
assertEquals("All replicas not corrupted", REPL_FACTOR, blockFilesCorrupted);
|
||||
|
||||
FSDataInputStream dis = cluster.getFileSystem().open(path);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect((int)FILE_LENGTH);
|
||||
boolean sawException = false;
|
||||
try {
|
||||
dis.read(buf);
|
||||
} catch (ChecksumException ex) {
|
||||
sawException = true;
|
||||
}
|
||||
|
||||
assertTrue(sawException);
|
||||
assertEquals(0, buf.position());
|
||||
assertEquals(buf.capacity(), buf.limit());
|
||||
|
||||
dis = cluster.getFileSystem().open(path);
|
||||
buf.position(3);
|
||||
buf.limit(25);
|
||||
sawException = false;
|
||||
try {
|
||||
dis.read(buf);
|
||||
} catch (ChecksumException ex) {
|
||||
sawException = true;
|
||||
}
|
||||
|
||||
assertTrue(sawException);
|
||||
assertEquals(3, buf.position());
|
||||
assertEquals(25, buf.limit());
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBothOldAndNewShortCircuitConfigured() throws Exception {
|
||||
final short REPL_FACTOR = 1;
|
||||
final int FILE_LENGTH = 512;
|
||||
Assume.assumeTrue(null == DomainSocket.getLoadingFailureReason());
|
||||
TemporarySocketDirectory socketDir = new TemporarySocketDirectory();
|
||||
HdfsConfiguration conf = getConfiguration(socketDir);
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
socketDir.close();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
|
||||
Path path = new Path("/foo");
|
||||
byte orig[] = new byte[FILE_LENGTH];
|
||||
for (int i = 0; i < orig.length; i++) {
|
||||
orig[i] = (byte)(i%10);
|
||||
}
|
||||
FSDataOutputStream fos = fs.create(path, (short)1);
|
||||
fos.write(orig);
|
||||
fos.close();
|
||||
DFSTestUtil.waitReplication(fs, path, REPL_FACTOR);
|
||||
FSDataInputStream fis = cluster.getFileSystem().open(path);
|
||||
byte buf[] = new byte[FILE_LENGTH];
|
||||
IOUtils.readFully(fis, buf, 0, FILE_LENGTH);
|
||||
fis.close();
|
||||
Assert.assertArrayEquals(orig, buf);
|
||||
Arrays.equals(orig, buf);
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
|
@ -61,7 +61,7 @@ public class TestClientBlockVerification {
|
|||
util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024));
|
||||
util.readAndCheckEOS(reader, FILE_SIZE_K * 1024, true);
|
||||
verify(reader).sendReadResult(Status.CHECKSUM_OK);
|
||||
reader.close();
|
||||
reader.close(null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -76,7 +76,7 @@ public class TestClientBlockVerification {
|
|||
// We asked the blockreader for the whole file, and only read
|
||||
// half of it, so no CHECKSUM_OK
|
||||
verify(reader, never()).sendReadResult(Status.CHECKSUM_OK);
|
||||
reader.close();
|
||||
reader.close(null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -92,7 +92,7 @@ public class TestClientBlockVerification {
|
|||
// And read half the file
|
||||
util.readAndCheckEOS(reader, FILE_SIZE_K * 1024 / 2, true);
|
||||
verify(reader).sendReadResult(Status.CHECKSUM_OK);
|
||||
reader.close();
|
||||
reader.close(null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -111,7 +111,7 @@ public class TestClientBlockVerification {
|
|||
util.getBlockReader(testBlock, startOffset, length));
|
||||
util.readAndCheckEOS(reader, length, true);
|
||||
verify(reader).sendReadResult(Status.CHECKSUM_OK);
|
||||
reader.close();
|
||||
reader.close(null, null);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,28 +18,20 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.spy;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Matchers;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -55,59 +47,31 @@ public class TestConnCache {
|
|||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
static final int FILE_SIZE = 3 * BLOCK_SIZE;
|
||||
final static int CACHE_SIZE = 4;
|
||||
final static long CACHE_EXPIRY_MS = 200;
|
||||
static Configuration conf = null;
|
||||
static MiniDFSCluster cluster = null;
|
||||
static FileSystem fs = null;
|
||||
static SocketCache cache;
|
||||
|
||||
static final Path testFile = new Path("/testConnCache.dat");
|
||||
static byte authenticData[] = null;
|
||||
|
||||
static BlockReaderTestUtil util = null;
|
||||
|
||||
|
||||
/**
|
||||
* A mock Answer to remember the BlockReader used.
|
||||
*
|
||||
* It verifies that all invocation to DFSInputStream.getBlockReader()
|
||||
* use the same socket.
|
||||
* use the same peer.
|
||||
*/
|
||||
private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
|
||||
public RemoteBlockReader2 reader = null;
|
||||
private Socket sock = null;
|
||||
private Peer peer = null;
|
||||
|
||||
@Override
|
||||
public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable {
|
||||
RemoteBlockReader2 prevReader = reader;
|
||||
reader = (RemoteBlockReader2) invocation.callRealMethod();
|
||||
if (sock == null) {
|
||||
sock = reader.dnSock;
|
||||
if (peer == null) {
|
||||
peer = reader.getPeer();
|
||||
} else if (prevReader != null) {
|
||||
assertSame("DFSInputStream should use the same socket",
|
||||
sock, reader.dnSock);
|
||||
Assert.assertSame("DFSInputStream should use the same peer",
|
||||
peer, reader.getPeer());
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
final int REPLICATION_FACTOR = 1;
|
||||
|
||||
/* create a socket cache. There is only one socket cache per jvm */
|
||||
cache = SocketCache.getInstance(CACHE_SIZE, CACHE_EXPIRY_MS);
|
||||
|
||||
util = new BlockReaderTestUtil(REPLICATION_FACTOR);
|
||||
cluster = util.getCluster();
|
||||
conf = util.getConf();
|
||||
fs = cluster.getFileSystem();
|
||||
|
||||
authenticData = util.writeFile(testFile, FILE_SIZE / 1024);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* (Optionally) seek to position, read and verify data.
|
||||
*
|
||||
|
@ -117,9 +81,10 @@ public class TestConnCache {
|
|||
long pos,
|
||||
byte[] buffer,
|
||||
int offset,
|
||||
int length)
|
||||
int length,
|
||||
byte[] authenticData)
|
||||
throws IOException {
|
||||
assertTrue("Test buffer too small", buffer.length >= offset + length);
|
||||
Assert.assertTrue("Test buffer too small", buffer.length >= offset + length);
|
||||
|
||||
if (pos >= 0)
|
||||
in.seek(pos);
|
||||
|
@ -129,7 +94,7 @@ public class TestConnCache {
|
|||
|
||||
while (length > 0) {
|
||||
int cnt = in.read(buffer, offset, length);
|
||||
assertTrue("Error in read", cnt > 0);
|
||||
Assert.assertTrue("Error in read", cnt > 0);
|
||||
offset += cnt;
|
||||
length -= cnt;
|
||||
}
|
||||
|
@ -144,116 +109,23 @@ public class TestConnCache {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the SocketCache itself.
|
||||
*/
|
||||
@Test
|
||||
public void testSocketCache() throws Exception {
|
||||
// Make a client
|
||||
InetSocketAddress nnAddr =
|
||||
new InetSocketAddress("localhost", cluster.getNameNodePort());
|
||||
DFSClient client = new DFSClient(nnAddr, conf);
|
||||
|
||||
// Find out the DN addr
|
||||
LocatedBlock block =
|
||||
client.getNamenode().getBlockLocations(
|
||||
testFile.toString(), 0, FILE_SIZE)
|
||||
.getLocatedBlocks().get(0);
|
||||
DataNode dn = util.getDataNode(block);
|
||||
InetSocketAddress dnAddr = dn.getXferAddress();
|
||||
|
||||
|
||||
// Make some sockets to the DN
|
||||
Socket[] dnSockets = new Socket[CACHE_SIZE];
|
||||
for (int i = 0; i < dnSockets.length; ++i) {
|
||||
dnSockets[i] = client.socketFactory.createSocket(
|
||||
dnAddr.getAddress(), dnAddr.getPort());
|
||||
}
|
||||
|
||||
|
||||
// Insert a socket to the NN
|
||||
Socket nnSock = new Socket(nnAddr.getAddress(), nnAddr.getPort());
|
||||
cache.put(nnSock, null);
|
||||
assertSame("Read the write", nnSock, cache.get(nnAddr).sock);
|
||||
cache.put(nnSock, null);
|
||||
|
||||
// Insert DN socks
|
||||
for (Socket dnSock : dnSockets) {
|
||||
cache.put(dnSock, null);
|
||||
}
|
||||
|
||||
assertEquals("NN socket evicted", null, cache.get(nnAddr));
|
||||
assertTrue("Evicted socket closed", nnSock.isClosed());
|
||||
|
||||
// Lookup the DN socks
|
||||
for (Socket dnSock : dnSockets) {
|
||||
assertEquals("Retrieve cached sockets", dnSock, cache.get(dnAddr).sock);
|
||||
dnSock.close();
|
||||
}
|
||||
|
||||
assertEquals("Cache is empty", 0, cache.size());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test the SocketCache expiry.
|
||||
* Verify that socket cache entries expire after the set
|
||||
* expiry time.
|
||||
*/
|
||||
@Test
|
||||
public void testSocketCacheExpiry() throws Exception {
|
||||
// Make a client
|
||||
InetSocketAddress nnAddr =
|
||||
new InetSocketAddress("localhost", cluster.getNameNodePort());
|
||||
DFSClient client = new DFSClient(nnAddr, conf);
|
||||
|
||||
// Find out the DN addr
|
||||
LocatedBlock block =
|
||||
client.getNamenode().getBlockLocations(
|
||||
testFile.toString(), 0, FILE_SIZE)
|
||||
.getLocatedBlocks().get(0);
|
||||
DataNode dn = util.getDataNode(block);
|
||||
InetSocketAddress dnAddr = dn.getXferAddress();
|
||||
|
||||
|
||||
// Make some sockets to the DN and put in cache
|
||||
Socket[] dnSockets = new Socket[CACHE_SIZE];
|
||||
for (int i = 0; i < dnSockets.length; ++i) {
|
||||
dnSockets[i] = client.socketFactory.createSocket(
|
||||
dnAddr.getAddress(), dnAddr.getPort());
|
||||
cache.put(dnSockets[i], null);
|
||||
}
|
||||
|
||||
// Client side still has the sockets cached
|
||||
assertEquals(CACHE_SIZE, client.socketCache.size());
|
||||
|
||||
//sleep for a second and see if it expired
|
||||
Thread.sleep(CACHE_EXPIRY_MS + 1000);
|
||||
|
||||
// Client side has no sockets cached
|
||||
assertEquals(0, client.socketCache.size());
|
||||
|
||||
//sleep for another second and see if
|
||||
//the daemon thread runs fine on empty cache
|
||||
Thread.sleep(CACHE_EXPIRY_MS + 1000);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Read a file served entirely from one DN. Seek around and read from
|
||||
* different offsets. And verify that they all use the same socket.
|
||||
*
|
||||
* @throws java.io.IOException
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testReadFromOneDN() throws IOException {
|
||||
LOG.info("Starting testReadFromOneDN()");
|
||||
public void testReadFromOneDN() throws Exception {
|
||||
BlockReaderTestUtil util = new BlockReaderTestUtil(1,
|
||||
new HdfsConfiguration());
|
||||
final Path testFile = new Path("/testConnCache.dat");
|
||||
byte authenticData[] = util.writeFile(testFile, FILE_SIZE / 1024);
|
||||
DFSClient client = new DFSClient(
|
||||
new InetSocketAddress("localhost", cluster.getNameNodePort()), conf);
|
||||
DFSInputStream in = spy(client.open(testFile.toString()));
|
||||
new InetSocketAddress("localhost",
|
||||
util.getCluster().getNameNodePort()), util.getConf());
|
||||
DFSInputStream in = Mockito.spy(client.open(testFile.toString()));
|
||||
LOG.info("opened " + testFile.toString());
|
||||
|
||||
byte[] dataBuf = new byte[BLOCK_SIZE];
|
||||
|
||||
MockGetBlockReader answer = new MockGetBlockReader();
|
||||
|
@ -270,18 +142,15 @@ public class TestConnCache {
|
|||
Matchers.anyString());
|
||||
|
||||
// Initial read
|
||||
pread(in, 0, dataBuf, 0, dataBuf.length);
|
||||
pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);
|
||||
// Read again and verify that the socket is the same
|
||||
pread(in, FILE_SIZE - dataBuf.length, dataBuf, 0, dataBuf.length);
|
||||
pread(in, 1024, dataBuf, 0, dataBuf.length);
|
||||
pread(in, -1, dataBuf, 0, dataBuf.length); // No seek; just read
|
||||
pread(in, 64, dataBuf, 0, dataBuf.length / 2);
|
||||
pread(in, FILE_SIZE - dataBuf.length, dataBuf, 0, dataBuf.length,
|
||||
authenticData);
|
||||
pread(in, 1024, dataBuf, 0, dataBuf.length, authenticData);
|
||||
// No seek; just read
|
||||
pread(in, -1, dataBuf, 0, dataBuf.length, authenticData);
|
||||
pread(in, 64, dataBuf, 0, dataBuf.length / 2, authenticData);
|
||||
|
||||
in.close();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownCluster() throws Exception {
|
||||
util.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||
|
@ -70,7 +71,7 @@ public class TestDataTransferKeepalive {
|
|||
.numDataNodes(1).build();
|
||||
fs = cluster.getFileSystem();
|
||||
dfsClient = ((DistributedFileSystem)fs).dfs;
|
||||
dfsClient.socketCache.clear();
|
||||
dfsClient.peerCache.clear();
|
||||
|
||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||
dn = cluster.getDataNodes().get(0);
|
||||
|
@ -93,13 +94,13 @@ public class TestDataTransferKeepalive {
|
|||
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
|
||||
|
||||
// Clients that write aren't currently re-used.
|
||||
assertEquals(0, dfsClient.socketCache.size());
|
||||
assertEquals(0, dfsClient.peerCache.size());
|
||||
assertXceiverCount(0);
|
||||
|
||||
// Reads the file, so we should get a
|
||||
// cached socket, and should have an xceiver on the other side.
|
||||
DFSTestUtil.readFile(fs, TEST_FILE);
|
||||
assertEquals(1, dfsClient.socketCache.size());
|
||||
assertEquals(1, dfsClient.peerCache.size());
|
||||
assertXceiverCount(1);
|
||||
|
||||
// Sleep for a bit longer than the keepalive timeout
|
||||
|
@ -110,13 +111,13 @@ public class TestDataTransferKeepalive {
|
|||
// The socket is still in the cache, because we don't
|
||||
// notice that it's closed until we try to read
|
||||
// from it again.
|
||||
assertEquals(1, dfsClient.socketCache.size());
|
||||
assertEquals(1, dfsClient.peerCache.size());
|
||||
|
||||
// Take it out of the cache - reading should
|
||||
// give an EOF.
|
||||
Socket s = dfsClient.socketCache.get(dnAddr).sock;
|
||||
assertNotNull(s);
|
||||
assertEquals(-1, NetUtils.getInputStream(s).read());
|
||||
Peer peer = dfsClient.peerCache.get(dn.getDatanodeId(), false);
|
||||
assertNotNull(peer);
|
||||
assertEquals(-1, peer.getInputStream().read());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -175,14 +176,14 @@ public class TestDataTransferKeepalive {
|
|||
}
|
||||
|
||||
DFSClient client = ((DistributedFileSystem)fs).dfs;
|
||||
assertEquals(5, client.socketCache.size());
|
||||
assertEquals(5, client.peerCache.size());
|
||||
|
||||
// Let all the xceivers timeout
|
||||
Thread.sleep(1500);
|
||||
assertXceiverCount(0);
|
||||
|
||||
// Client side still has the sockets cached
|
||||
assertEquals(5, client.socketCache.size());
|
||||
assertEquals(5, client.peerCache.size());
|
||||
|
||||
// Reading should not throw an exception.
|
||||
DFSTestUtil.readFile(fs, TEST_FILE);
|
||||
|
|
|
@ -0,0 +1,62 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* This class tests disabling client connection caching in a single node
|
||||
* mini-cluster.
|
||||
*/
|
||||
public class TestDisableConnCache {
|
||||
static final Log LOG = LogFactory.getLog(TestDisableConnCache.class);
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
static final int FILE_SIZE = 3 * BLOCK_SIZE;
|
||||
|
||||
/**
|
||||
* Test that the socket cache can be disabled by setting the capacity to
|
||||
* 0. Regression test for HDFS-3365.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testDisableCache() throws Exception {
|
||||
HdfsConfiguration confWithoutCache = new HdfsConfiguration();
|
||||
// Configure a new instance with no peer caching, ensure that it doesn't
|
||||
// cache anything
|
||||
confWithoutCache.setInt(
|
||||
DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 0);
|
||||
BlockReaderTestUtil util = new BlockReaderTestUtil(1, confWithoutCache);
|
||||
final Path testFile = new Path("/testConnCache.dat");
|
||||
util.writeFile(testFile, FILE_SIZE / 1024);
|
||||
FileSystem fsWithoutCache = FileSystem.newInstance(util.getConf());
|
||||
try {
|
||||
DFSTestUtil.readFile(fsWithoutCache, testFile);
|
||||
assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.peerCache.size());
|
||||
} finally {
|
||||
fsWithoutCache.close();
|
||||
util.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,127 @@
|
|||
/**
|
||||
* 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.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.TemporarySocketDirectory;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestFileInputStreamCache {
|
||||
static final Log LOG = LogFactory.getLog(TestFileInputStreamCache.class);
|
||||
|
||||
@Test
|
||||
public void testCreateAndDestroy() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(10, 1000);
|
||||
cache.close();
|
||||
}
|
||||
|
||||
private static class TestFileDescriptorPair {
|
||||
TemporarySocketDirectory dir = new TemporarySocketDirectory();
|
||||
FileInputStream fis[];
|
||||
|
||||
public TestFileDescriptorPair() throws IOException {
|
||||
fis = new FileInputStream[2];
|
||||
for (int i = 0; i < 2; i++) {
|
||||
String name = dir.getDir() + "/file" + i;
|
||||
FileOutputStream fos = new FileOutputStream(name);
|
||||
fos.write(1);
|
||||
fos.close();
|
||||
fis[i] = new FileInputStream(name);
|
||||
}
|
||||
}
|
||||
|
||||
public FileInputStream[] getFileInputStreams() {
|
||||
return fis;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public boolean compareWith(FileInputStream other[]) {
|
||||
if ((other == null) || (fis == null)) {
|
||||
return other == fis;
|
||||
}
|
||||
if (fis.length != other.length) return false;
|
||||
for (int i = 0; i < fis.length; i++) {
|
||||
if (fis[i] != other[i]) return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddAndRetrieve() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(1, 1000000);
|
||||
DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8080, 9090, 7070);
|
||||
ExtendedBlock block = new ExtendedBlock("poolid", 123);
|
||||
TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
cache.put(dnId, block, pair.getFileInputStreams());
|
||||
FileInputStream fis[] = cache.get(dnId, block);
|
||||
Assert.assertTrue(pair.compareWith(fis));
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExpiry() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(1, 10);
|
||||
DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8080, 9090, 7070);
|
||||
ExtendedBlock block = new ExtendedBlock("poolid", 123);
|
||||
TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
cache.put(dnId, block, pair.getFileInputStreams());
|
||||
Thread.sleep(cache.getExpiryTimeMs() * 100);
|
||||
FileInputStream fis[] = cache.get(dnId, block);
|
||||
Assert.assertNull(fis);
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEviction() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(1, 10000000);
|
||||
DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8080, 9090, 7070);
|
||||
ExtendedBlock block = new ExtendedBlock("poolid", 123);
|
||||
TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
cache.put(dnId, block, pair.getFileInputStreams());
|
||||
DatanodeID dnId2 = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8081, 9091, 7071);
|
||||
TestFileDescriptorPair pair2 = new TestFileDescriptorPair();
|
||||
cache.put(dnId2, block, pair2.getFileInputStreams());
|
||||
FileInputStream fis[] = cache.get(dnId, block);
|
||||
Assert.assertNull(fis);
|
||||
FileInputStream fis2[] = cache.get(dnId2, block);
|
||||
Assert.assertTrue(pair2.compareWith(fis2));
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
}
|
|
@ -17,53 +17,30 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestParallelRead extends TestParallelReadUtil {
|
||||
|
||||
@BeforeClass
|
||||
static public void setupCluster() throws Exception {
|
||||
setupCluster(DEFAULT_REPLICATION_FACTOR, new HdfsConfiguration());
|
||||
// This is a test of the normal (TCP) read path. For this reason, we turn
|
||||
// off both short-circuit local reads and UNIX domain socket data traffic.
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
|
||||
false);
|
||||
// dfs.domain.socket.path should be ignored because the previous two keys
|
||||
// were set to false. This is a regression test for HDFS-4473.
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "/will/not/be/created");
|
||||
|
||||
setupCluster(DEFAULT_REPLICATION_FACTOR, conf);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void teardownCluster() throws Exception {
|
||||
TestParallelReadUtil.teardownCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Do parallel read several times with different number of files and threads.
|
||||
*
|
||||
* Note that while this is the only "test" in a junit sense, we're actually
|
||||
* dispatching a lot more. Failures in the other methods (and other threads)
|
||||
* need to be manually collected, which is inconvenient.
|
||||
*/
|
||||
@Test
|
||||
public void testParallelReadCopying() throws IOException {
|
||||
runTestWorkload(new CopyingReadWorkerHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelReadByteBuffer() throws IOException {
|
||||
runTestWorkload(new DirectReadWorkerHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelReadMixed() throws IOException {
|
||||
runTestWorkload(new MixedWorkloadHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelNoChecksums() throws IOException {
|
||||
verifyChecksums = false;
|
||||
runTestWorkload(new MixedWorkloadHelper());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -32,12 +32,18 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Driver class for testing the use of DFSInputStream by multiple concurrent
|
||||
* readers, using the different read APIs. See subclasses for the actual test
|
||||
* cases.
|
||||
* readers, using the different read APIs.
|
||||
*
|
||||
* This class is marked as @Ignore so that junit doesn't try to execute the
|
||||
* tests in here directly. They are executed from subclasses.
|
||||
*/
|
||||
@Ignore
|
||||
public class TestParallelReadUtil {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(TestParallelReadUtil.class);
|
||||
|
@ -388,4 +394,31 @@ public class TestParallelReadUtil {
|
|||
util.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Do parallel read several times with different number of files and threads.
|
||||
*
|
||||
* Note that while this is the only "test" in a junit sense, we're actually
|
||||
* dispatching a lot more. Failures in the other methods (and other threads)
|
||||
* need to be manually collected, which is inconvenient.
|
||||
*/
|
||||
@Test
|
||||
public void testParallelReadCopying() throws IOException {
|
||||
runTestWorkload(new CopyingReadWorkerHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelReadByteBuffer() throws IOException {
|
||||
runTestWorkload(new DirectReadWorkerHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelReadMixed() throws IOException {
|
||||
runTestWorkload(new MixedWorkloadHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelNoChecksums() throws IOException {
|
||||
verifyChecksums = false;
|
||||
runTestWorkload(new MixedWorkloadHelper());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,25 +17,25 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestParallelLocalRead extends TestParallelReadUtil {
|
||||
|
||||
public class TestParallelShortCircuitLegacyRead extends TestParallelReadUtil {
|
||||
@BeforeClass
|
||||
static public void setupCluster() throws Exception {
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "");
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
false);
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
UserGroupInformation.getCurrentUser().getShortUserName());
|
||||
|
||||
DomainSocket.disableBindPathValidation();
|
||||
setupCluster(1, conf);
|
||||
}
|
||||
|
||||
|
@ -43,26 +43,4 @@ public class TestParallelLocalRead extends TestParallelReadUtil {
|
|||
static public void teardownCluster() throws Exception {
|
||||
TestParallelReadUtil.teardownCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Do parallel read several times with different number of files and threads.
|
||||
*
|
||||
* Note that while this is the only "test" in a junit sense, we're actually
|
||||
* dispatching a lot more. Failures in the other methods (and other threads)
|
||||
* need to be manually collected, which is inconvenient.
|
||||
*/
|
||||
@Test
|
||||
public void testParallelReadCopying() throws IOException {
|
||||
runTestWorkload(new CopyingReadWorkerHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelReadByteBuffer() throws IOException {
|
||||
runTestWorkload(new DirectReadWorkerHelper());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelReadMixed() throws IOException {
|
||||
runTestWorkload(new MixedWorkloadHelper());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/**
|
||||
* 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.File;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import static org.hamcrest.CoreMatchers.*;
|
||||
|
||||
public class TestParallelShortCircuitRead extends TestParallelReadUtil {
|
||||
private static TemporarySocketDirectory sockDir;
|
||||
|
||||
@BeforeClass
|
||||
static public void setupCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
|
||||
DomainSocket.disableBindPathValidation();
|
||||
setupCluster(1, conf);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void teardownCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
sockDir.close();
|
||||
TestParallelReadUtil.teardownCluster();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/**
|
||||
* 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.File;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import static org.hamcrest.CoreMatchers.*;
|
||||
|
||||
public class TestParallelShortCircuitReadNoChecksum extends TestParallelReadUtil {
|
||||
private static TemporarySocketDirectory sockDir;
|
||||
|
||||
@BeforeClass
|
||||
static public void setupCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
|
||||
DomainSocket.disableBindPathValidation();
|
||||
setupCluster(1, conf);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void teardownCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
sockDir.close();
|
||||
TestParallelReadUtil.teardownCluster();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
/**
|
||||
* 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.File;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import static org.hamcrest.CoreMatchers.*;
|
||||
|
||||
/**
|
||||
* This class tests short-circuit local reads without any FileInputStream or
|
||||
* Socket caching. This is a regression test for HDFS-4417.
|
||||
*/
|
||||
public class TestParallelShortCircuitReadUnCached extends TestParallelReadUtil {
|
||||
private static TemporarySocketDirectory sockDir;
|
||||
|
||||
@BeforeClass
|
||||
static public void setupCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(),
|
||||
"TestParallelShortCircuitReadUnCached._PORT.sock").getAbsolutePath());
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
|
||||
// We want to test reading from stale sockets.
|
||||
conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY, 1);
|
||||
conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
|
||||
5 * 60 * 1000);
|
||||
conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 32);
|
||||
// Avoid using the FileInputStreamCache.
|
||||
conf.setInt(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY, 0);
|
||||
DomainSocket.disableBindPathValidation();
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
setupCluster(1, conf);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void teardownCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
sockDir.close();
|
||||
TestParallelReadUtil.teardownCluster();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* 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.File;
|
||||
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import static org.hamcrest.CoreMatchers.*;
|
||||
|
||||
public class TestParallelUnixDomainRead extends TestParallelReadUtil {
|
||||
private static TemporarySocketDirectory sockDir;
|
||||
|
||||
@BeforeClass
|
||||
static public void setupCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
|
||||
DomainSocket.disableBindPathValidation();
|
||||
setupCluster(1, conf);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void teardownCluster() throws Exception {
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return;
|
||||
sockDir.close();
|
||||
TestParallelReadUtil.teardownCluster();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,288 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.channels.ReadableByteChannel;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import com.google.common.collect.HashMultiset;
|
||||
|
||||
public class TestPeerCache {
|
||||
static final Log LOG = LogFactory.getLog(TestPeerCache.class);
|
||||
|
||||
private static class FakePeer implements Peer {
|
||||
private boolean closed = false;
|
||||
private final boolean hasDomain;
|
||||
|
||||
private DatanodeID dnId;
|
||||
|
||||
public FakePeer(DatanodeID dnId, boolean hasDomain) {
|
||||
this.dnId = dnId;
|
||||
this.hasDomain = hasDomain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableByteChannel getInputStreamChannel() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReadTimeout(int timeoutMs) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getReceiveBufferSize() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getTcpNoDelay() throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteTimeout(int timeoutMs) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return closed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
closed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRemoteAddressString() {
|
||||
return dnId.getInfoAddr();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLocalAddressString() {
|
||||
return "127.0.0.1:123";
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream getOutputStream() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FakePeer(dnId=" + dnId + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DomainSocket getDomainSocket() {
|
||||
if (!hasDomain) return null;
|
||||
// Return a mock which throws an exception whenever any function is
|
||||
// called.
|
||||
return Mockito.mock(DomainSocket.class,
|
||||
new Answer<Object>() {
|
||||
@Override
|
||||
public Object answer(InvocationOnMock invocation)
|
||||
throws Throwable {
|
||||
throw new RuntimeException("injected fault.");
|
||||
} });
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (!(o instanceof FakePeer)) return false;
|
||||
FakePeer other = (FakePeer)o;
|
||||
return hasDomain == other.hasDomain &&
|
||||
dnId.equals(other.dnId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return dnId.hashCode() ^ (hasDomain ? 1 : 0);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddAndRetrieve() throws Exception {
|
||||
PeerCache cache = new PeerCache(3, 100000);
|
||||
DatanodeID dnId = new DatanodeID("192.168.0.1",
|
||||
"fakehostname", "fake_storage_id",
|
||||
100, 101, 102);
|
||||
FakePeer peer = new FakePeer(dnId, false);
|
||||
cache.put(dnId, peer);
|
||||
assertTrue(!peer.isClosed());
|
||||
assertEquals(1, cache.size());
|
||||
assertEquals(peer, cache.get(dnId, false));
|
||||
assertEquals(0, cache.size());
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExpiry() throws Exception {
|
||||
final int CAPACITY = 3;
|
||||
final int EXPIRY_PERIOD = 10;
|
||||
PeerCache cache = new PeerCache(CAPACITY, EXPIRY_PERIOD);
|
||||
DatanodeID dnIds[] = new DatanodeID[CAPACITY];
|
||||
FakePeer peers[] = new FakePeer[CAPACITY];
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
dnIds[i] = new DatanodeID("192.168.0.1",
|
||||
"fakehostname_" + i, "fake_storage_id",
|
||||
100, 101, 102);
|
||||
peers[i] = new FakePeer(dnIds[i], false);
|
||||
}
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
cache.put(dnIds[i], peers[i]);
|
||||
}
|
||||
|
||||
// Wait for the peers to expire
|
||||
Thread.sleep(EXPIRY_PERIOD * 50);
|
||||
assertEquals(0, cache.size());
|
||||
|
||||
// make sure that the peers were closed when they were expired
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
assertTrue(peers[i].isClosed());
|
||||
}
|
||||
|
||||
// sleep for another second and see if
|
||||
// the daemon thread runs fine on empty cache
|
||||
Thread.sleep(EXPIRY_PERIOD * 50);
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEviction() throws Exception {
|
||||
final int CAPACITY = 3;
|
||||
PeerCache cache = new PeerCache(CAPACITY, 100000);
|
||||
DatanodeID dnIds[] = new DatanodeID[CAPACITY + 1];
|
||||
FakePeer peers[] = new FakePeer[CAPACITY + 1];
|
||||
for (int i = 0; i < dnIds.length; ++i) {
|
||||
dnIds[i] = new DatanodeID("192.168.0.1",
|
||||
"fakehostname_" + i, "fake_storage_id_" + i,
|
||||
100, 101, 102);
|
||||
peers[i] = new FakePeer(dnIds[i], false);
|
||||
}
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
cache.put(dnIds[i], peers[i]);
|
||||
}
|
||||
// Check that the peers are cached
|
||||
assertEquals(CAPACITY, cache.size());
|
||||
|
||||
// Add another entry and check that the first entry was evicted
|
||||
cache.put(dnIds[CAPACITY], peers[CAPACITY]);
|
||||
assertEquals(CAPACITY, cache.size());
|
||||
assertSame(null, cache.get(dnIds[0], false));
|
||||
|
||||
// Make sure that the other entries are still there
|
||||
for (int i = 1; i < CAPACITY; ++i) {
|
||||
Peer peer = cache.get(dnIds[i], false);
|
||||
assertSame(peers[i], peer);
|
||||
assertTrue(!peer.isClosed());
|
||||
peer.close();
|
||||
}
|
||||
assertEquals(1, cache.size());
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiplePeersWithSameKey() throws Exception {
|
||||
final int CAPACITY = 3;
|
||||
PeerCache cache = new PeerCache(CAPACITY, 100000);
|
||||
DatanodeID dnId = new DatanodeID("192.168.0.1",
|
||||
"fakehostname", "fake_storage_id",
|
||||
100, 101, 102);
|
||||
HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
FakePeer peer = new FakePeer(dnId, false);
|
||||
peers.add(peer);
|
||||
cache.put(dnId, peer);
|
||||
}
|
||||
// Check that all of the peers ended up in the cache
|
||||
assertEquals(CAPACITY, cache.size());
|
||||
while (!peers.isEmpty()) {
|
||||
Peer peer = cache.get(dnId, false);
|
||||
assertTrue(peer != null);
|
||||
assertTrue(!peer.isClosed());
|
||||
peers.remove(peer);
|
||||
}
|
||||
assertEquals(0, cache.size());
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDomainSocketPeers() throws Exception {
|
||||
final int CAPACITY = 3;
|
||||
PeerCache cache = new PeerCache(CAPACITY, 100000);
|
||||
DatanodeID dnId = new DatanodeID("192.168.0.1",
|
||||
"fakehostname", "fake_storage_id",
|
||||
100, 101, 102);
|
||||
HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
FakePeer peer = new FakePeer(dnId, i == CAPACITY - 1);
|
||||
peers.add(peer);
|
||||
cache.put(dnId, peer);
|
||||
}
|
||||
// Check that all of the peers ended up in the cache
|
||||
assertEquals(CAPACITY, cache.size());
|
||||
// Test that get(requireDomainPeer=true) finds the peer with the
|
||||
// domain socket.
|
||||
Peer peer = cache.get(dnId, true);
|
||||
assertTrue(peer.getDomainSocket() != null);
|
||||
peers.remove(peer);
|
||||
// Test that get(requireDomainPeer=true) returns null when there are
|
||||
// no more peers with domain sockets.
|
||||
peer = cache.get(dnId, true);
|
||||
assertTrue(peer == null);
|
||||
// Check that all of the other peers ended up in the cache.
|
||||
while (!peers.isEmpty()) {
|
||||
peer = cache.get(dnId, false);
|
||||
assertTrue(peer != null);
|
||||
assertTrue(!peer.isClosed());
|
||||
peers.remove(peer);
|
||||
}
|
||||
assertEquals(0, cache.size());
|
||||
cache.close();
|
||||
}
|
||||
}
|
|
@ -21,10 +21,13 @@ import static org.junit.Assert.assertTrue;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.net.URI;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
|
@ -32,7 +35,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
|
@ -40,15 +42,21 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import static org.hamcrest.CoreMatchers.*;
|
||||
|
||||
/**
|
||||
* Test for short circuit read functionality using {@link BlockReaderLocal}.
|
||||
|
@ -58,9 +66,24 @@ import org.junit.Test;
|
|||
* system.
|
||||
*/
|
||||
public class TestShortCircuitLocalRead {
|
||||
private static TemporarySocketDirectory sockDir;
|
||||
|
||||
static final String DIR = "/" + TestShortCircuitLocalRead.class.getSimpleName() + "/";
|
||||
@BeforeClass
|
||||
public static void init() {
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
DomainSocket.disableBindPathValidation();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() throws IOException {
|
||||
sockDir.close();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
}
|
||||
|
||||
static final long seed = 0xDEADBEEFL;
|
||||
static final int blockSize = 5120;
|
||||
boolean simulatedStorage = false;
|
||||
|
@ -84,7 +107,9 @@ public class TestShortCircuitLocalRead {
|
|||
for (int idx = 0; idx < len; idx++) {
|
||||
if (expected[from + idx] != actual[idx]) {
|
||||
Assert.fail(message + " byte " + (from + idx) + " differs. expected "
|
||||
+ expected[from + idx] + " actual " + actual[idx]);
|
||||
+ expected[from + idx] + " actual " + actual[idx] +
|
||||
"\nexpected: " + StringUtils.byteToHexString(expected, from, from + len) +
|
||||
"\nactual: " + StringUtils.byteToHexString(actual, 0, len));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -96,11 +121,13 @@ public class TestShortCircuitLocalRead {
|
|||
/** Check file content, reading as user {@code readingUser} */
|
||||
static void checkFileContent(URI uri, Path name, byte[] expected,
|
||||
int readOffset, String readingUser, Configuration conf,
|
||||
boolean shortCircuitFails)
|
||||
boolean legacyShortCircuitFails)
|
||||
throws IOException, InterruptedException {
|
||||
// Ensure short circuit is enabled
|
||||
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
|
||||
assertTrue(fs.getClient().getShortCircuitLocalReads());
|
||||
if (legacyShortCircuitFails) {
|
||||
assertTrue(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
|
||||
FSDataInputStream stm = fs.open(name);
|
||||
byte[] actual = new byte[expected.length-readOffset];
|
||||
|
@ -127,9 +154,8 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
checkData(actual, readOffset, expected, "Read 3");
|
||||
|
||||
if (shortCircuitFails) {
|
||||
// short circuit should be disabled due to failure
|
||||
assertFalse(fs.getClient().getShortCircuitLocalReads());
|
||||
if (legacyShortCircuitFails) {
|
||||
assertFalse(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
@ -145,11 +171,13 @@ public class TestShortCircuitLocalRead {
|
|||
/** Check the file content, reading as user {@code readingUser} */
|
||||
static void checkFileContentDirect(URI uri, Path name, byte[] expected,
|
||||
int readOffset, String readingUser, Configuration conf,
|
||||
boolean shortCircuitFails)
|
||||
boolean legacyShortCircuitFails)
|
||||
throws IOException, InterruptedException {
|
||||
// Ensure short circuit is enabled
|
||||
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
|
||||
assertTrue(fs.getClient().getShortCircuitLocalReads());
|
||||
if (legacyShortCircuitFails) {
|
||||
assertTrue(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
|
||||
HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name);
|
||||
|
||||
|
@ -180,33 +208,45 @@ public class TestShortCircuitLocalRead {
|
|||
nread += nbytes;
|
||||
}
|
||||
checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3");
|
||||
if (shortCircuitFails) {
|
||||
// short circuit should be disabled due to failure
|
||||
assertFalse(fs.getClient().getShortCircuitLocalReads());
|
||||
if (legacyShortCircuitFails) {
|
||||
assertFalse(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
||||
public void doTestShortCircuitReadLegacy(boolean ignoreChecksum, int size,
|
||||
int readOffset, String shortCircuitUser, String readingUser,
|
||||
boolean legacyShortCircuitFails) throws IOException, InterruptedException {
|
||||
doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
|
||||
shortCircuitUser, readingUser, legacyShortCircuitFails);
|
||||
}
|
||||
|
||||
public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
|
||||
int readOffset) throws IOException, InterruptedException {
|
||||
String shortCircuitUser = getCurrentUser();
|
||||
doTestShortCircuitRead(ignoreChecksum, size, readOffset, shortCircuitUser,
|
||||
shortCircuitUser, false);
|
||||
doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
|
||||
null, getCurrentUser(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that file data can be read by reading the block file
|
||||
* directly from the local store.
|
||||
*/
|
||||
public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
|
||||
public void doTestShortCircuitReadImpl(boolean ignoreChecksum, int size,
|
||||
int readOffset, String shortCircuitUser, String readingUser,
|
||||
boolean shortCircuitFails) throws IOException, InterruptedException {
|
||||
boolean legacyShortCircuitFails) throws IOException, InterruptedException {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
ignoreChecksum);
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
shortCircuitUser);
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(),
|
||||
"TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
|
||||
if (shortCircuitUser != null) {
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
shortCircuitUser);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
|
||||
}
|
||||
if (simulatedStorage) {
|
||||
SimulatedFSDataset.setFactory(conf);
|
||||
}
|
||||
|
@ -228,9 +268,9 @@ public class TestShortCircuitLocalRead {
|
|||
|
||||
URI uri = cluster.getURI();
|
||||
checkFileContent(uri, file1, fileData, readOffset, readingUser, conf,
|
||||
shortCircuitFails);
|
||||
legacyShortCircuitFails);
|
||||
checkFileContentDirect(uri, file1, fileData, readOffset, readingUser,
|
||||
conf, shortCircuitFails);
|
||||
conf, legacyShortCircuitFails);
|
||||
} finally {
|
||||
fs.close();
|
||||
cluster.shutdown();
|
||||
|
@ -255,6 +295,12 @@ public class TestShortCircuitLocalRead {
|
|||
doTestShortCircuitRead(true, 13, 5);
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testLocalReadLegacy() throws Exception {
|
||||
doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(),
|
||||
getCurrentUser(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Try a short circuit from a reader that is not allowed to
|
||||
* to use short circuit. The test ensures reader falls back to non
|
||||
|
@ -262,7 +308,7 @@ public class TestShortCircuitLocalRead {
|
|||
*/
|
||||
@Test(timeout=10000)
|
||||
public void testLocalReadFallback() throws Exception {
|
||||
doTestShortCircuitRead(true, 13, 0, getCurrentUser(), "notallowed", true);
|
||||
doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(), "notallowed", true);
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
|
@ -276,7 +322,7 @@ public class TestShortCircuitLocalRead {
|
|||
doTestShortCircuitRead(false, 10*blockSize+100, 777);
|
||||
doTestShortCircuitRead(true, 10*blockSize+100, 777);
|
||||
}
|
||||
|
||||
|
||||
private ClientDatanodeProtocol getProxy(UserGroupInformation ugi,
|
||||
final DatanodeID dnInfo, final Configuration conf) throws IOException,
|
||||
InterruptedException {
|
||||
|
@ -301,21 +347,15 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testGetBlockLocalPathInfo() throws IOException, InterruptedException {
|
||||
public void testDeprecatedGetBlockLocalPathInfoRpc()
|
||||
throws IOException, InterruptedException {
|
||||
final Configuration conf = new Configuration();
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
"alloweduser1,alloweduser2");
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
|
||||
.format(true).build();
|
||||
cluster.waitActive();
|
||||
final DataNode dn = cluster.getDataNodes().get(0);
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
try {
|
||||
DFSTestUtil.createFile(fs, new Path("/tmp/x"), 16, (short) 1, 23);
|
||||
UserGroupInformation aUgi1 =
|
||||
UserGroupInformation.createRemoteUser("alloweduser1");
|
||||
UserGroupInformation aUgi2 =
|
||||
UserGroupInformation.createRemoteUser("alloweduser2");
|
||||
LocatedBlocks lb = cluster.getNameNode().getRpcServer()
|
||||
.getBlockLocations("/tmp/x", 0, 16);
|
||||
// Create a new block object, because the block inside LocatedBlock at
|
||||
|
@ -323,29 +363,11 @@ public class TestShortCircuitLocalRead {
|
|||
ExtendedBlock blk = new ExtendedBlock(lb.get(0).getBlock());
|
||||
Token<BlockTokenIdentifier> token = lb.get(0).getBlockToken();
|
||||
final DatanodeInfo dnInfo = lb.get(0).getLocations()[0];
|
||||
ClientDatanodeProtocol proxy = getProxy(aUgi1, dnInfo, conf);
|
||||
// This should succeed
|
||||
BlockLocalPathInfo blpi = proxy.getBlockLocalPathInfo(blk, token);
|
||||
Assert.assertEquals(
|
||||
DataNodeTestUtils.getFSDataset(dn).getBlockLocalPathInfo(blk).getBlockPath(),
|
||||
blpi.getBlockPath());
|
||||
|
||||
// Try with the other allowed user
|
||||
proxy = getProxy(aUgi2, dnInfo, conf);
|
||||
|
||||
// This should succeed as well
|
||||
blpi = proxy.getBlockLocalPathInfo(blk, token);
|
||||
Assert.assertEquals(
|
||||
DataNodeTestUtils.getFSDataset(dn).getBlockLocalPathInfo(blk).getBlockPath(),
|
||||
blpi.getBlockPath());
|
||||
|
||||
// Now try with a disallowed user
|
||||
UserGroupInformation bUgi = UserGroupInformation
|
||||
.createRemoteUser("notalloweduser");
|
||||
proxy = getProxy(bUgi, dnInfo, conf);
|
||||
ClientDatanodeProtocol proxy =
|
||||
DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf, 60000, false);
|
||||
try {
|
||||
proxy.getBlockLocalPathInfo(blk, token);
|
||||
Assert.fail("The call should have failed as " + bUgi.getShortUserName()
|
||||
Assert.fail("The call should have failed as this user "
|
||||
+ " is not allowed to call getBlockLocalPathInfo");
|
||||
} catch (IOException ex) {
|
||||
Assert.assertTrue(ex.getMessage().contains(
|
||||
|
@ -363,8 +385,9 @@ public class TestShortCircuitLocalRead {
|
|||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
getCurrentUser());
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
"/tmp/testSkipWithVerifyChecksum._PORT");
|
||||
DomainSocket.disableBindPathValidation();
|
||||
if (simulatedStorage) {
|
||||
SimulatedFSDataset.setFactory(conf);
|
||||
}
|
||||
|
@ -402,6 +425,88 @@ public class TestShortCircuitLocalRead {
|
|||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHandleTruncatedBlockFile() throws IOException {
|
||||
MiniDFSCluster cluster = null;
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
"/tmp/testHandleTruncatedBlockFile._PORT");
|
||||
conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
|
||||
final Path TEST_PATH = new Path("/a");
|
||||
final Path TEST_PATH2 = new Path("/b");
|
||||
final long RANDOM_SEED = 4567L;
|
||||
final long RANDOM_SEED2 = 4568L;
|
||||
FSDataInputStream fsIn = null;
|
||||
final int TEST_LENGTH = 3456;
|
||||
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||
TEST_LENGTH, (short)1, RANDOM_SEED);
|
||||
DFSTestUtil.createFile(fs, TEST_PATH2,
|
||||
TEST_LENGTH, (short)1, RANDOM_SEED2);
|
||||
fsIn = cluster.getFileSystem().open(TEST_PATH2);
|
||||
byte original[] = new byte[TEST_LENGTH];
|
||||
IOUtils.readFully(fsIn, original, 0, TEST_LENGTH);
|
||||
fsIn.close();
|
||||
fsIn = null;
|
||||
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);
|
||||
}
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
||||
File dataFile = MiniDFSCluster.getBlockFile(0, block);
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
RandomAccessFile raf = null;
|
||||
try {
|
||||
raf = new RandomAccessFile(dataFile, "rw");
|
||||
raf.setLength(0);
|
||||
} finally {
|
||||
if (raf != null) raf.close();
|
||||
}
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(false).build();
|
||||
cluster.waitActive();
|
||||
fs = cluster.getFileSystem();
|
||||
fsIn = fs.open(TEST_PATH);
|
||||
try {
|
||||
byte buf[] = new byte[100];
|
||||
fsIn.seek(2000);
|
||||
fsIn.readFully(buf, 0, buf.length);
|
||||
Assert.fail("shouldn't be able to read from corrupt 0-length " +
|
||||
"block file.");
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.error("caught exception ", e);
|
||||
}
|
||||
fsIn.close();
|
||||
fsIn = null;
|
||||
|
||||
// We should still be able to read the other file.
|
||||
// This is important because it indicates that we detected that the
|
||||
// previous block was corrupt, rather than blaming the problem on
|
||||
// communication.
|
||||
fsIn = fs.open(TEST_PATH2);
|
||||
byte buf[] = new byte[original.length];
|
||||
fsIn.readFully(buf, 0, buf.length);
|
||||
TestBlockReaderLocal.assertArrayRegionsEqual(original, 0, buf, 0,
|
||||
original.length);
|
||||
fsIn.close();
|
||||
fsIn = null;
|
||||
} finally {
|
||||
if (fsIn != null) fsIn.close();
|
||||
if (cluster != null) cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to run benchmarks between short circuit read vs regular read with
|
||||
|
@ -424,6 +529,8 @@ public class TestShortCircuitLocalRead {
|
|||
// Setup create a file
|
||||
final Configuration conf = new Configuration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
"/tmp/TestShortCircuitLocalRead._PORT");
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
checksum);
|
||||
|
||||
|
|
|
@ -1,171 +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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.spy;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Matchers;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
/**
|
||||
* This class tests the client connection caching in a single node
|
||||
* mini-cluster.
|
||||
*/
|
||||
public class TestSocketCache {
|
||||
static final Log LOG = LogFactory.getLog(TestSocketCache.class);
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
static final int FILE_SIZE = 3 * BLOCK_SIZE;
|
||||
final static int CACHE_SIZE = 4;
|
||||
final static long CACHE_EXPIRY_MS = 200;
|
||||
static Configuration conf = null;
|
||||
static MiniDFSCluster cluster = null;
|
||||
static FileSystem fs = null;
|
||||
static SocketCache cache;
|
||||
|
||||
static final Path testFile = new Path("/testConnCache.dat");
|
||||
static byte authenticData[] = null;
|
||||
|
||||
static BlockReaderTestUtil util = null;
|
||||
|
||||
|
||||
/**
|
||||
* A mock Answer to remember the BlockReader used.
|
||||
*
|
||||
* It verifies that all invocation to DFSInputStream.getBlockReader()
|
||||
* use the same socket.
|
||||
*/
|
||||
private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
|
||||
public RemoteBlockReader2 reader = null;
|
||||
private Socket sock = null;
|
||||
|
||||
@Override
|
||||
public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable {
|
||||
RemoteBlockReader2 prevReader = reader;
|
||||
reader = (RemoteBlockReader2) invocation.callRealMethod();
|
||||
if (sock == null) {
|
||||
sock = reader.dnSock;
|
||||
} else if (prevReader != null) {
|
||||
assertSame("DFSInputStream should use the same socket",
|
||||
sock, reader.dnSock);
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
final int REPLICATION_FACTOR = 1;
|
||||
|
||||
HdfsConfiguration confWithoutCache = new HdfsConfiguration();
|
||||
confWithoutCache.setInt(
|
||||
DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 0);
|
||||
util = new BlockReaderTestUtil(REPLICATION_FACTOR, confWithoutCache);
|
||||
cluster = util.getCluster();
|
||||
conf = util.getConf();
|
||||
|
||||
authenticData = util.writeFile(testFile, FILE_SIZE / 1024);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* (Optionally) seek to position, read and verify data.
|
||||
*
|
||||
* Seek to specified position if pos is non-negative.
|
||||
*/
|
||||
private void pread(DFSInputStream in,
|
||||
long pos,
|
||||
byte[] buffer,
|
||||
int offset,
|
||||
int length)
|
||||
throws IOException {
|
||||
assertTrue("Test buffer too small", buffer.length >= offset + length);
|
||||
|
||||
if (pos >= 0)
|
||||
in.seek(pos);
|
||||
|
||||
LOG.info("Reading from file of size " + in.getFileLength() +
|
||||
" at offset " + in.getPos());
|
||||
|
||||
while (length > 0) {
|
||||
int cnt = in.read(buffer, offset, length);
|
||||
assertTrue("Error in read", cnt > 0);
|
||||
offset += cnt;
|
||||
length -= cnt;
|
||||
}
|
||||
|
||||
// Verify
|
||||
for (int i = 0; i < length; ++i) {
|
||||
byte actual = buffer[i];
|
||||
byte expect = authenticData[(int)pos + i];
|
||||
assertEquals("Read data mismatch at file offset " + (pos + i) +
|
||||
". Expects " + expect + "; got " + actual,
|
||||
actual, expect);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test that the socket cache can be disabled by setting the capacity to
|
||||
* 0. Regression test for HDFS-3365.
|
||||
*/
|
||||
@Test
|
||||
public void testDisableCache() throws IOException {
|
||||
LOG.info("Starting testDisableCache()");
|
||||
|
||||
// Configure a new instance with no caching, ensure that it doesn't
|
||||
// cache anything
|
||||
|
||||
FileSystem fsWithoutCache = FileSystem.newInstance(conf);
|
||||
try {
|
||||
DFSTestUtil.readFile(fsWithoutCache, testFile);
|
||||
assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.socketCache.size());
|
||||
} finally {
|
||||
fsWithoutCache.close();
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownCluster() throws Exception {
|
||||
util.shutdown();
|
||||
}
|
||||
}
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.DFSClient;
|
|||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
|
@ -145,8 +146,9 @@ public class TestBlockTokenWithDFS {
|
|||
String file = BlockReaderFactory.getFileName(targetAddr,
|
||||
"test-blockpoolid", block.getBlockId());
|
||||
blockReader = BlockReaderFactory.newBlockReader(
|
||||
conf, s, file, block,
|
||||
lblock.getBlockToken(), 0, -1, null);
|
||||
conf, file, block, lblock.getBlockToken(), 0, -1,
|
||||
true, "TestBlockTokenWithDFS", TcpPeerServer.peerFromSocket(s),
|
||||
nodes[0], null, false);
|
||||
|
||||
} catch (IOException ex) {
|
||||
if (ex instanceof InvalidBlockTokenException) {
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
@ -962,6 +963,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|||
public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileInputStream[] getShortCircuitFdsForRead(ExtendedBlock block)
|
||||
throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
|
||||
|
|
|
@ -32,11 +32,13 @@ import java.util.Map;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -280,10 +282,11 @@ public class TestDataNodeVolumeFailure {
|
|||
String file = BlockReaderFactory.getFileName(targetAddr,
|
||||
"test-blockpoolid",
|
||||
block.getBlockId());
|
||||
BlockReaderFactory.newBlockReader(conf, s, file, block, lblock
|
||||
.getBlockToken(), 0, -1, null);
|
||||
|
||||
// nothing - if it fails - it will throw and exception
|
||||
BlockReader blockReader =
|
||||
BlockReaderFactory.newBlockReader(conf, file, block,
|
||||
lblock.getBlockToken(), 0, -1, true, "TestDataNodeVolumeFailure",
|
||||
TcpPeerServer.peerFromSocket(s), datanode, null, false);
|
||||
blockReader.close(null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -73,6 +73,8 @@
|
|||
<item name="C API libhdfs" href="hadoop-project-dist/hadoop-hdfs/LibHdfs.html"/>
|
||||
<item name="WebHDFS REST API" href="hadoop-project-dist/hadoop-hdfs/WebHDFS.html"/>
|
||||
<item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
|
||||
<item name="Short Circuit Local Reads"
|
||||
href="hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html"/>
|
||||
</menu>
|
||||
|
||||
<menu name="MapReduce" inherit="top">
|
||||
|
|
Loading…
Reference in New Issue