Skip to content

Commit

Permalink
HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS…
Browse files Browse the repository at this point in the history
…-8283. Contributed by Jing Zhao.
  • Loading branch information
Jing9 committed May 1, 2015
1 parent 87e9978 commit 98a6176
Show file tree
Hide file tree
Showing 4 changed files with 29 additions and 17 deletions.
3 changes: 3 additions & 0 deletions hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Expand Up @@ -592,6 +592,9 @@ Release 2.8.0 - UNRELEASED

HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. (clamb via wang)

HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283.
(jing9)

Release 2.7.1 - UNRELEASED

INCOMPATIBLE CHANGES
Expand Down
Expand Up @@ -762,7 +762,7 @@ public synchronized void close() throws IOException {

protected synchronized void closeImpl() throws IOException {
if (isClosed()) {
streamer.getLastException().check();
streamer.getLastException().check(true);
return;
}

Expand Down
Expand Up @@ -174,31 +174,37 @@ private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam)
packets.clear();
}

static class LastException {
private Throwable thrown;
static class LastExceptionInStreamer {
private IOException thrown;

synchronized void set(Throwable t) {
Preconditions.checkNotNull(t);
Preconditions.checkState(thrown == null);
this.thrown = t;
assert t != null;
this.thrown = t instanceof IOException ?
(IOException) t : new IOException(t);
}

synchronized void clear() {
thrown = null;
}

/** Check if there already is an exception. */
synchronized void check() throws IOException {
synchronized void check(boolean resetToNull) throws IOException {
if (thrown != null) {
throw new IOException(thrown);
if (LOG.isTraceEnabled()) {
// wrap and print the exception to know when the check is called
LOG.trace("Got Exception while checking", new Throwable(thrown));
}
final IOException e = thrown;
if (resetToNull) {
thrown = null;
}
throw e;
}
}

synchronized void throwException4Close() throws IOException {
check();
final IOException ioe = new ClosedChannelException();
thrown = ioe;
throw ioe;
check(false);
throw new ClosedChannelException();
}
}

Expand Down Expand Up @@ -234,7 +240,7 @@ synchronized void throwException4Close() throws IOException {
private long lastQueuedSeqno = -1;
private long lastAckedSeqno = -1;
private long bytesCurBlock = 0; // bytes written in current block
private final LastException lastException = new LastException();
private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
private Socket s;

private final DFSClient dfsClient;
Expand Down Expand Up @@ -1741,7 +1747,7 @@ boolean getAppendChunk(){
/**
* @return the last exception
*/
LastException getLastException(){
LastExceptionInStreamer getLastException(){
return lastException;
}

Expand Down
Expand Up @@ -28,6 +28,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
Expand Down Expand Up @@ -65,9 +66,10 @@ public void testCloseTwice() throws IOException {
DataStreamer streamer = (DataStreamer) Whitebox
.getInternalState(dos, "streamer");
@SuppressWarnings("unchecked")
AtomicReference<IOException> ex = (AtomicReference<IOException>) Whitebox
LastExceptionInStreamer ex = (LastExceptionInStreamer) Whitebox
.getInternalState(streamer, "lastException");
Assert.assertEquals(null, ex.get());
Throwable thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
Assert.assertNull(thrown);

dos.close();

Expand All @@ -78,7 +80,8 @@ public void testCloseTwice() throws IOException {
} catch (IOException e) {
Assert.assertEquals(e, dummy);
}
Assert.assertEquals(null, ex.get());
thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
Assert.assertNull(thrown);
dos.close();
}

Expand Down

0 comments on commit 98a6176

Please sign in to comment.