|
@@ -24,17 +24,15 @@ import java.io.FileInputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
-import java.net.SocketException;
|
|
|
|
-import java.nio.channels.AsynchronousCloseException;
|
|
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.ReadableByteChannel;
|
|
import java.nio.channels.ReadableByteChannel;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
-import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
|
|
import org.apache.commons.lang.SystemUtils;
|
|
import org.apache.commons.lang.SystemUtils;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.util.NativeCodeLoader;
|
|
import org.apache.hadoop.util.NativeCodeLoader;
|
|
|
|
+import org.apache.hadoop.util.CloseableReferenceCount;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
@@ -132,104 +130,14 @@ public class DomainSocket implements Closeable {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Tracks the reference count of the file descriptor, and also whether it is
|
|
|
|
- * open or closed.
|
|
|
|
|
|
+ * The socket reference count and closed bit.
|
|
*/
|
|
*/
|
|
- 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;
|
|
|
|
|
|
+ final CloseableReferenceCount refCount;
|
|
|
|
|
|
/**
|
|
/**
|
|
* The file descriptor associated with this UNIX domain socket.
|
|
* The file descriptor associated with this UNIX domain socket.
|
|
*/
|
|
*/
|
|
- private final int fd;
|
|
|
|
|
|
+ final int fd;
|
|
|
|
|
|
/**
|
|
/**
|
|
* The path associated with this UNIX domain socket.
|
|
* The path associated with this UNIX domain socket.
|
|
@@ -252,13 +160,21 @@ public class DomainSocket implements Closeable {
|
|
private final DomainChannel channel = new DomainChannel();
|
|
private final DomainChannel channel = new DomainChannel();
|
|
|
|
|
|
private DomainSocket(String path, int fd) {
|
|
private DomainSocket(String path, int fd) {
|
|
- this.status = new Status();
|
|
|
|
|
|
+ this.refCount = new CloseableReferenceCount();
|
|
this.fd = fd;
|
|
this.fd = fd;
|
|
this.path = path;
|
|
this.path = path;
|
|
}
|
|
}
|
|
|
|
|
|
private static native int bind0(String path) throws IOException;
|
|
private static native int bind0(String path) throws IOException;
|
|
|
|
|
|
|
|
+ private void unreference(boolean checkClosed) throws ClosedChannelException {
|
|
|
|
+ if (checkClosed) {
|
|
|
|
+ refCount.unreferenceCheckClosed();
|
|
|
|
+ } else {
|
|
|
|
+ refCount.unreference();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create a new DomainSocket listening on the given path.
|
|
* Create a new DomainSocket listening on the given path.
|
|
*
|
|
*
|
|
@@ -308,14 +224,14 @@ public class DomainSocket implements Closeable {
|
|
* @throws SocketTimeoutException If the accept timed out.
|
|
* @throws SocketTimeoutException If the accept timed out.
|
|
*/
|
|
*/
|
|
public DomainSocket accept() throws IOException {
|
|
public DomainSocket accept() throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
DomainSocket ret = new DomainSocket(path, accept0(fd));
|
|
DomainSocket ret = new DomainSocket(path, accept0(fd));
|
|
exc = false;
|
|
exc = false;
|
|
return ret;
|
|
return ret;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -335,14 +251,14 @@ public class DomainSocket implements Closeable {
|
|
return new DomainSocket(path, fd);
|
|
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 true if the file descriptor is currently open.
|
|
|
|
+ *
|
|
|
|
+ * @return True if the file descriptor is currently open.
|
|
|
|
+ */
|
|
|
|
+ public boolean isOpen() {
|
|
|
|
+ return refCount.isOpen();
|
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
/**
|
|
* @return The socket path.
|
|
* @return The socket path.
|
|
@@ -381,20 +297,20 @@ public class DomainSocket implements Closeable {
|
|
throws IOException;
|
|
throws IOException;
|
|
|
|
|
|
public void setAttribute(int type, int size) throws IOException {
|
|
public void setAttribute(int type, int size) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
setAttribute0(fd, type, size);
|
|
setAttribute0(fd, type, size);
|
|
exc = false;
|
|
exc = false;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
private native int getAttribute0(int fd, int type) throws IOException;
|
|
private native int getAttribute0(int fd, int type) throws IOException;
|
|
|
|
|
|
public int getAttribute(int type) throws IOException {
|
|
public int getAttribute(int type) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
int attribute;
|
|
int attribute;
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
@@ -402,7 +318,7 @@ public class DomainSocket implements Closeable {
|
|
exc = false;
|
|
exc = false;
|
|
return attribute;
|
|
return attribute;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -419,9 +335,9 @@ public class DomainSocket implements Closeable {
|
|
@Override
|
|
@Override
|
|
public void close() throws IOException {
|
|
public void close() throws IOException {
|
|
// Set the closed bit on this DomainSocket
|
|
// Set the closed bit on this DomainSocket
|
|
- int refCount;
|
|
|
|
|
|
+ int count;
|
|
try {
|
|
try {
|
|
- refCount = status.setClosed();
|
|
|
|
|
|
+ count = refCount.setClosed();
|
|
} catch (ClosedChannelException e) {
|
|
} catch (ClosedChannelException e) {
|
|
// Someone else already closed the DomainSocket.
|
|
// Someone else already closed the DomainSocket.
|
|
return;
|
|
return;
|
|
@@ -429,7 +345,7 @@ public class DomainSocket implements Closeable {
|
|
// Wait for all references to go away
|
|
// Wait for all references to go away
|
|
boolean didShutdown = false;
|
|
boolean didShutdown = false;
|
|
boolean interrupted = false;
|
|
boolean interrupted = false;
|
|
- while (refCount > 0) {
|
|
|
|
|
|
+ while (count > 0) {
|
|
if (!didShutdown) {
|
|
if (!didShutdown) {
|
|
try {
|
|
try {
|
|
// Calling shutdown on the socket will interrupt blocking system
|
|
// Calling shutdown on the socket will interrupt blocking system
|
|
@@ -446,7 +362,7 @@ public class DomainSocket implements Closeable {
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
interrupted = true;
|
|
interrupted = true;
|
|
}
|
|
}
|
|
- refCount = status.getReferenceCount();
|
|
|
|
|
|
+ count = refCount.getReferenceCount();
|
|
}
|
|
}
|
|
|
|
|
|
// At this point, nobody has a reference to the file descriptor,
|
|
// At this point, nobody has a reference to the file descriptor,
|
|
@@ -478,13 +394,13 @@ public class DomainSocket implements Closeable {
|
|
*/
|
|
*/
|
|
public void sendFileDescriptors(FileDescriptor descriptors[],
|
|
public void sendFileDescriptors(FileDescriptor descriptors[],
|
|
byte jbuf[], int offset, int length) throws IOException {
|
|
byte jbuf[], int offset, int length) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
sendFileDescriptors0(fd, descriptors, jbuf, offset, length);
|
|
sendFileDescriptors0(fd, descriptors, jbuf, offset, length);
|
|
exc = false;
|
|
exc = false;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -515,14 +431,14 @@ public class DomainSocket implements Closeable {
|
|
*/
|
|
*/
|
|
public int receiveFileDescriptors(FileDescriptor[] descriptors,
|
|
public int receiveFileDescriptors(FileDescriptor[] descriptors,
|
|
byte jbuf[], int offset, int length) throws IOException {
|
|
byte jbuf[], int offset, int length) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
int nBytes = receiveFileDescriptors0(fd, descriptors, jbuf, offset, length);
|
|
int nBytes = receiveFileDescriptors0(fd, descriptors, jbuf, offset, length);
|
|
exc = false;
|
|
exc = false;
|
|
return nBytes;
|
|
return nBytes;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -539,7 +455,7 @@ public class DomainSocket implements Closeable {
|
|
for (int i = 0; i < streams.length; i++) {
|
|
for (int i = 0; i < streams.length; i++) {
|
|
streams[i] = null;
|
|
streams[i] = null;
|
|
}
|
|
}
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
try {
|
|
try {
|
|
int ret = receiveFileDescriptors0(fd, descriptors, buf, offset, length);
|
|
int ret = receiveFileDescriptors0(fd, descriptors, buf, offset, length);
|
|
for (int i = 0, j = 0; i < descriptors.length; i++) {
|
|
for (int i = 0, j = 0; i < descriptors.length; i++) {
|
|
@@ -569,7 +485,7 @@ public class DomainSocket implements Closeable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- status.unreference(!success);
|
|
|
|
|
|
+ unreference(!success);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -593,7 +509,7 @@ public class DomainSocket implements Closeable {
|
|
public class DomainInputStream extends InputStream {
|
|
public class DomainInputStream extends InputStream {
|
|
@Override
|
|
@Override
|
|
public int read() throws IOException {
|
|
public int read() throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
byte b[] = new byte[1];
|
|
byte b[] = new byte[1];
|
|
@@ -601,33 +517,33 @@ public class DomainSocket implements Closeable {
|
|
exc = false;
|
|
exc = false;
|
|
return (ret >= 0) ? b[0] : -1;
|
|
return (ret >= 0) ? b[0] : -1;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int read(byte b[], int off, int len) throws IOException {
|
|
public int read(byte b[], int off, int len) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
int nRead = DomainSocket.readArray0(DomainSocket.this.fd, b, off, len);
|
|
int nRead = DomainSocket.readArray0(DomainSocket.this.fd, b, off, len);
|
|
exc = false;
|
|
exc = false;
|
|
return nRead;
|
|
return nRead;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int available() throws IOException {
|
|
public int available() throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
int nAvailable = DomainSocket.available0(DomainSocket.this.fd);
|
|
int nAvailable = DomainSocket.available0(DomainSocket.this.fd);
|
|
exc = false;
|
|
exc = false;
|
|
return nAvailable;
|
|
return nAvailable;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -649,7 +565,7 @@ public class DomainSocket implements Closeable {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void write(int val) throws IOException {
|
|
public void write(int val) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
byte b[] = new byte[1];
|
|
byte b[] = new byte[1];
|
|
@@ -657,19 +573,19 @@ public class DomainSocket implements Closeable {
|
|
DomainSocket.writeArray0(DomainSocket.this.fd, b, 0, 1);
|
|
DomainSocket.writeArray0(DomainSocket.this.fd, b, 0, 1);
|
|
exc = false;
|
|
exc = false;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void write(byte[] b, int off, int len) throws IOException {
|
|
public void write(byte[] b, int off, int len) throws IOException {
|
|
- status.reference();
|
|
|
|
- boolean exc = true;
|
|
|
|
|
|
+ refCount.reference();
|
|
|
|
+ boolean exc = true;
|
|
try {
|
|
try {
|
|
DomainSocket.writeArray0(DomainSocket.this.fd, b, off, len);
|
|
DomainSocket.writeArray0(DomainSocket.this.fd, b, off, len);
|
|
exc = false;
|
|
exc = false;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -688,7 +604,7 @@ public class DomainSocket implements Closeable {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int read(ByteBuffer dst) throws IOException {
|
|
public int read(ByteBuffer dst) throws IOException {
|
|
- status.reference();
|
|
|
|
|
|
+ refCount.reference();
|
|
boolean exc = true;
|
|
boolean exc = true;
|
|
try {
|
|
try {
|
|
int nread = 0;
|
|
int nread = 0;
|
|
@@ -710,7 +626,7 @@ public class DomainSocket implements Closeable {
|
|
exc = false;
|
|
exc = false;
|
|
return nread;
|
|
return nread;
|
|
} finally {
|
|
} finally {
|
|
- status.unreference(exc);
|
|
|
|
|
|
+ unreference(exc);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|