HDFS-7515. Fix new findbugs warnings in hadoop-hdfs. Contributed by Haohui Mai.

This commit is contained in:
Haohui Mai 2014-12-11 12:36:13 -08:00
parent 614b6afea4
commit b9f6d0c956
33 changed files with 188 additions and 200 deletions

View File

@ -572,6 +572,8 @@ Release 2.7.0 - UNRELEASED
HDFS-7475. Make TestLazyPersistFiles#testLazyPersistBlocksAreSaved HDFS-7475. Make TestLazyPersistFiles#testLazyPersistBlocksAreSaved
deterministic. (Xiaoyu Yao via Arpit Agarwal) deterministic. (Xiaoyu Yao via Arpit Agarwal)
HDFS-7515. Fix new findbugs warnings in hadoop-hdfs. (wheat9)
Release 2.6.1 - UNRELEASED Release 2.6.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -668,7 +668,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
Peer peer = null; Peer peer = null;
try { try {
curPeer = nextTcpPeer(); curPeer = nextTcpPeer();
if (curPeer == null) break;
if (curPeer.fromCache) remainingCacheTries--; if (curPeer.fromCache) remainingCacheTries--;
peer = curPeer.peer; peer = curPeer.peer;
blockReader = getRemoteBlockReader(peer); blockReader = getRemoteBlockReader(peer);
@ -699,7 +698,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
} }
} }
} }
return null;
} }
public static class BlockReaderPeer { public static class BlockReaderPeer {

View File

@ -39,6 +39,7 @@ import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.HadoopIllegalArgumentException;
@ -241,8 +242,6 @@ public class DFSOutputStream extends FSOutputSummer
/** /**
* Create a new packet. * Create a new packet.
* *
* @param pktSize maximum size of the packet,
* including checksum data and actual data.
* @param chunksPerPkt maximum number of chunks per packet. * @param chunksPerPkt maximum number of chunks per packet.
* @param offsetInBlock offset in bytes into the HDFS block. * @param offsetInBlock offset in bytes into the HDFS block.
*/ */
@ -405,7 +404,8 @@ public class DFSOutputStream extends FSOutputSummer
private String[] favoredNodes; private String[] favoredNodes;
volatile boolean hasError = false; volatile boolean hasError = false;
volatile int errorIndex = -1; volatile int errorIndex = -1;
volatile int restartingNodeIndex = -1; // Restarting node index // Restarting node index
AtomicInteger restartingNodeIndex = new AtomicInteger(-1);
private long restartDeadline = 0; // Deadline of DN restart private long restartDeadline = 0; // Deadline of DN restart
private BlockConstructionStage stage; // block construction stage private BlockConstructionStage stage; // block construction stage
private long bytesSent = 0; // number of bytes that've been sent private long bytesSent = 0; // number of bytes that've been sent
@ -556,7 +556,7 @@ public class DFSOutputStream extends FSOutputSummer
try { try {
// process datanode IO errors if any // process datanode IO errors if any
boolean doSleep = false; boolean doSleep = false;
if (hasError && (errorIndex >= 0 || restartingNodeIndex >= 0)) { if (hasError && (errorIndex >= 0 || restartingNodeIndex.get() >= 0)) {
doSleep = processDatanodeError(); doSleep = processDatanodeError();
} }
@ -699,7 +699,7 @@ public class DFSOutputStream extends FSOutputSummer
} }
} catch (Throwable e) { } catch (Throwable e) {
// Log warning if there was a real error. // Log warning if there was a real error.
if (restartingNodeIndex == -1) { if (restartingNodeIndex.get() == -1) {
DFSClient.LOG.warn("DataStreamer Exception", e); DFSClient.LOG.warn("DataStreamer Exception", e);
} }
if (e instanceof IOException) { if (e instanceof IOException) {
@ -708,7 +708,7 @@ public class DFSOutputStream extends FSOutputSummer
setLastException(new IOException("DataStreamer Exception: ",e)); setLastException(new IOException("DataStreamer Exception: ",e));
} }
hasError = true; hasError = true;
if (errorIndex == -1 && restartingNodeIndex == -1) { if (errorIndex == -1 && restartingNodeIndex.get() == -1) {
// Not a datanode issue // Not a datanode issue
streamerClosed = true; streamerClosed = true;
} }
@ -806,7 +806,7 @@ public class DFSOutputStream extends FSOutputSummer
/** Set the restarting node index. Called by responder */ /** Set the restarting node index. Called by responder */
synchronized void setRestartingNodeIndex(int idx) { synchronized void setRestartingNodeIndex(int idx) {
restartingNodeIndex = idx; restartingNodeIndex.set(idx);
// If the data streamer has already set the primary node // If the data streamer has already set the primary node
// bad, clear it. It is likely that the write failed due to // bad, clear it. It is likely that the write failed due to
// the DN shutdown. Even if it was a real failure, the pipeline // the DN shutdown. Even if it was a real failure, the pipeline
@ -821,7 +821,7 @@ public class DFSOutputStream extends FSOutputSummer
*/ */
synchronized void tryMarkPrimaryDatanodeFailed() { synchronized void tryMarkPrimaryDatanodeFailed() {
// There should be no existing error and no ongoing restart. // There should be no existing error and no ongoing restart.
if ((errorIndex == -1) && (restartingNodeIndex == -1)) { if ((errorIndex == -1) && (restartingNodeIndex.get() == -1)) {
errorIndex = 0; errorIndex = 0;
} }
} }
@ -962,7 +962,7 @@ public class DFSOutputStream extends FSOutputSummer
synchronized (dataQueue) { synchronized (dataQueue) {
dataQueue.notifyAll(); dataQueue.notifyAll();
} }
if (restartingNodeIndex == -1) { if (restartingNodeIndex.get() == -1) {
DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception " DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
+ " for block " + block, e); + " for block " + block, e);
} }
@ -1186,7 +1186,7 @@ public class DFSOutputStream extends FSOutputSummer
// Sleep before reconnect if a dn is restarting. // Sleep before reconnect if a dn is restarting.
// This process will be repeated until the deadline or the datanode // This process will be repeated until the deadline or the datanode
// starts back up. // starts back up.
if (restartingNodeIndex >= 0) { if (restartingNodeIndex.get() >= 0) {
// 4 seconds or the configured deadline period, whichever is shorter. // 4 seconds or the configured deadline period, whichever is shorter.
// This is the retry interval and recovery will be retried in this // This is the retry interval and recovery will be retried in this
// interval until timeout or success. // interval until timeout or success.
@ -1196,7 +1196,7 @@ public class DFSOutputStream extends FSOutputSummer
Thread.sleep(delay); Thread.sleep(delay);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
lastException.set(new IOException("Interrupted while waiting for " + lastException.set(new IOException("Interrupted while waiting for " +
"datanode to restart. " + nodes[restartingNodeIndex])); "datanode to restart. " + nodes[restartingNodeIndex.get()]));
streamerClosed = true; streamerClosed = true;
return false; return false;
} }
@ -1237,21 +1237,21 @@ public class DFSOutputStream extends FSOutputSummer
setPipeline(newnodes, newStorageTypes, newStorageIDs); setPipeline(newnodes, newStorageTypes, newStorageIDs);
// Just took care of a node error while waiting for a node restart // Just took care of a node error while waiting for a node restart
if (restartingNodeIndex >= 0) { if (restartingNodeIndex.get() >= 0) {
// If the error came from a node further away than the restarting // If the error came from a node further away than the restarting
// node, the restart must have been complete. // node, the restart must have been complete.
if (errorIndex > restartingNodeIndex) { if (errorIndex > restartingNodeIndex.get()) {
restartingNodeIndex = -1; restartingNodeIndex.set(-1);
} else if (errorIndex < restartingNodeIndex) { } else if (errorIndex < restartingNodeIndex.get()) {
// the node index has shifted. // the node index has shifted.
restartingNodeIndex--; restartingNodeIndex.decrementAndGet();
} else { } else {
// this shouldn't happen... // this shouldn't happen...
assert false; assert false;
} }
} }
if (restartingNodeIndex == -1) { if (restartingNodeIndex.get() == -1) {
hasError = false; hasError = false;
} }
lastException.set(null); lastException.set(null);
@ -1293,10 +1293,10 @@ public class DFSOutputStream extends FSOutputSummer
success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery); success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
} }
if (restartingNodeIndex >= 0) { if (restartingNodeIndex.get() >= 0) {
assert hasError == true; assert hasError == true;
// check errorIndex set above // check errorIndex set above
if (errorIndex == restartingNodeIndex) { if (errorIndex == restartingNodeIndex.get()) {
// ignore, if came from the restarting node // ignore, if came from the restarting node
errorIndex = -1; errorIndex = -1;
} }
@ -1306,8 +1306,8 @@ public class DFSOutputStream extends FSOutputSummer
} }
// expired. declare the restarting node dead // expired. declare the restarting node dead
restartDeadline = 0; restartDeadline = 0;
int expiredNodeIndex = restartingNodeIndex; int expiredNodeIndex = restartingNodeIndex.get();
restartingNodeIndex = -1; restartingNodeIndex.set(-1);
DFSClient.LOG.warn("Datanode did not restart in time: " + DFSClient.LOG.warn("Datanode did not restart in time: " +
nodes[expiredNodeIndex]); nodes[expiredNodeIndex]);
// Mark the restarting node as failed. If there is any other failed // Mark the restarting node as failed. If there is any other failed
@ -1459,7 +1459,7 @@ public class DFSOutputStream extends FSOutputSummer
// from the local datanode. Thus it is safe to treat this as a // from the local datanode. Thus it is safe to treat this as a
// regular node error. // regular node error.
if (PipelineAck.isRestartOOBStatus(pipelineStatus) && if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
restartingNodeIndex == -1) { restartingNodeIndex.get() == -1) {
checkRestart = true; checkRestart = true;
throw new IOException("A datanode is restarting."); throw new IOException("A datanode is restarting.");
} }
@ -1476,10 +1476,10 @@ public class DFSOutputStream extends FSOutputSummer
assert null == blockStream : "Previous blockStream unclosed"; assert null == blockStream : "Previous blockStream unclosed";
blockStream = out; blockStream = out;
result = true; // success result = true; // success
restartingNodeIndex = -1; restartingNodeIndex.set(-1);
hasError = false; hasError = false;
} catch (IOException ie) { } catch (IOException ie) {
if (restartingNodeIndex == -1) { if (restartingNodeIndex.get() == -1) {
DFSClient.LOG.info("Exception in createBlockOutputStream", ie); DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
} }
if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
@ -1511,10 +1511,10 @@ public class DFSOutputStream extends FSOutputSummer
if (checkRestart && shouldWaitForRestart(errorIndex)) { if (checkRestart && shouldWaitForRestart(errorIndex)) {
restartDeadline = dfsClient.getConf().datanodeRestartTimeout + restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
Time.now(); Time.now();
restartingNodeIndex = errorIndex; restartingNodeIndex.set(errorIndex);
errorIndex = -1; errorIndex = -1;
DFSClient.LOG.info("Waiting for the datanode to be restarted: " + DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
nodes[restartingNodeIndex]); nodes[restartingNodeIndex.get()]);
} }
hasError = true; hasError = true;
setLastException(ie); setLastException(ie);

View File

@ -233,6 +233,7 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
Preconditions.checkArgument(jid != null && Preconditions.checkArgument(jid != null &&
!jid.isEmpty(), !jid.isEmpty(),
"bad journal identifier: %s", jid); "bad journal identifier: %s", jid);
assert jid != null;
return new File(new File(dir), jid); return new File(new File(dir), jid);
} }

View File

@ -727,7 +727,7 @@ public abstract class Storage extends StorageInfo {
file.close(); file.close();
throw e; throw e;
} }
if (res != null && !deletionHookAdded) { if (!deletionHookAdded) {
// If the file existed prior to our startup, we didn't // If the file existed prior to our startup, we didn't
// call deleteOnExit above. But since we successfully locked // call deleteOnExit above. But since we successfully locked
// the dir, we can take care of cleaning it up. // the dir, we can take care of cleaning it up.

View File

@ -29,6 +29,8 @@ import java.io.FileOutputStream;
import java.io.FileWriter; import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.io.Writer;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.LinkedList; import java.util.LinkedList;
@ -836,9 +838,8 @@ class BlockReceiver implements Closeable {
LOG.warn("Failed to delete restart meta file: " + LOG.warn("Failed to delete restart meta file: " +
restartMeta.getPath()); restartMeta.getPath());
} }
FileWriter out = null; try (Writer out = new OutputStreamWriter(
try { new FileOutputStream(restartMeta), "UTF-8")) {
out = new FileWriter(restartMeta);
// write out the current time. // write out the current time.
out.write(Long.toString(Time.now() + restartBudget)); out.write(Long.toString(Time.now() + restartBudget));
out.flush(); out.flush();

View File

@ -580,7 +580,8 @@ public class DataNode extends ReconfigurableBase
try { try {
IOException ioe = ioExceptionFuture.get(); IOException ioe = ioExceptionFuture.get();
if (ioe != null) { if (ioe != null) {
errorMessageBuilder.append(String.format("FAILED TO ADD: %s: %s\n", errorMessageBuilder.append(
String.format("FAILED TO ADD: %s: %s%n",
volume, ioe.getMessage())); volume, ioe.getMessage()));
LOG.error("Failed to add volume: " + volume, ioe); LOG.error("Failed to add volume: " + volume, ioe);
} else { } else {
@ -588,8 +589,9 @@ public class DataNode extends ReconfigurableBase
LOG.info("Successfully added volume: " + volume); LOG.info("Successfully added volume: " + volume);
} }
} catch (Exception e) { } catch (Exception e) {
errorMessageBuilder.append(String.format("FAILED to ADD: %s: %s\n", errorMessageBuilder.append(
volume, e.getMessage())); String.format("FAILED to ADD: %s: %s%n", volume,
e.toString()));
} }
} }
} }

View File

@ -425,7 +425,7 @@ public class DataStorage extends Storage {
LOG.warn(String.format( LOG.warn(String.format(
"I/O error attempting to unlock storage directory %s.", "I/O error attempting to unlock storage directory %s.",
sd.getRoot()), e); sd.getRoot()), e);
errorMsgBuilder.append(String.format("Failed to remove %s: %s\n", errorMsgBuilder.append(String.format("Failed to remove %s: %s%n",
sd.getRoot(), e.getMessage())); sd.getRoot(), e.getMessage()));
} }
} }

View File

@ -22,10 +22,13 @@ import java.io.DataInputStream;
import java.io.File; import java.io.File;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.FileWriter; import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStreamWriter;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.io.Writer;
import java.util.Scanner; import java.util.Scanner;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
@ -186,7 +189,7 @@ class BlockPoolSlice {
Scanner sc; Scanner sc;
try { try {
sc = new Scanner(new File(currentDir, DU_CACHE_FILE)); sc = new Scanner(new File(currentDir, DU_CACHE_FILE), "UTF-8");
} catch (FileNotFoundException fnfe) { } catch (FileNotFoundException fnfe) {
return -1; return -1;
} }
@ -227,23 +230,18 @@ class BlockPoolSlice {
outFile.getParent()); outFile.getParent());
} }
FileWriter out = null;
try { try {
long used = getDfsUsed(); long used = getDfsUsed();
if (used > 0) { try (Writer out = new OutputStreamWriter(
out = new FileWriter(outFile); new FileOutputStream(outFile), "UTF-8")) {
// mtime is written last, so that truncated writes won't be valid. // mtime is written last, so that truncated writes won't be valid.
out.write(Long.toString(used) + " " + Long.toString(Time.now())); out.write(Long.toString(used) + " " + Long.toString(Time.now()));
out.flush(); out.flush();
out.close();
out = null;
} }
} catch (IOException ioe) { } catch (IOException ioe) {
// If write failed, the volume might be bad. Since the cache file is // If write failed, the volume might be bad. Since the cache file is
// not critical, log the error and continue. // not critical, log the error and continue.
FsDatasetImpl.LOG.warn("Failed to write dfsUsed to " + outFile, ioe); FsDatasetImpl.LOG.warn("Failed to write dfsUsed to " + outFile, ioe);
} finally {
IOUtils.cleanup(null, out);
} }
} }
@ -447,7 +445,7 @@ class BlockPoolSlice {
File.pathSeparator + "." + file.getName() + ".restart"); File.pathSeparator + "." + file.getName() + ".restart");
Scanner sc = null; Scanner sc = null;
try { try {
sc = new Scanner(restartMeta); sc = new Scanner(restartMeta, "UTF-8");
// The restart meta file exists // The restart meta file exists
if (sc.hasNextLong() && (sc.nextLong() > Time.now())) { if (sc.hasNextLong() && (sc.nextLong() > Time.now())) {
// It didn't expire. Load the replica as a RBW. // It didn't expire. Load the replica as a RBW.

View File

@ -769,7 +769,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
final byte[] crcs = new byte[checksum.getChecksumSize(data.length)]; final byte[] crcs = new byte[checksum.getChecksumSize(data.length)];
DataOutputStream metaOut = null; DataOutputStream metaOut = null;
InputStream dataIn = null;
try { try {
File parentFile = dstMeta.getParentFile(); File parentFile = dstMeta.getParentFile();
if (parentFile != null) { if (parentFile != null) {
@ -782,11 +781,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
new FileOutputStream(dstMeta), HdfsConstants.SMALL_BUFFER_SIZE)); new FileOutputStream(dstMeta), HdfsConstants.SMALL_BUFFER_SIZE));
BlockMetadataHeader.writeHeader(metaOut, checksum); BlockMetadataHeader.writeHeader(metaOut, checksum);
dataIn = isNativeIOAvailable ?
NativeIO.getShareDeleteFileInputStream(blockFile) :
new FileInputStream(blockFile);
int offset = 0; int offset = 0;
try (InputStream dataIn = isNativeIOAvailable ?
NativeIO.getShareDeleteFileInputStream(blockFile) :
new FileInputStream(blockFile)) {
for (int n; (n = dataIn.read(data, offset, data.length - offset)) != -1; ) { for (int n; (n = dataIn.read(data, offset, data.length - offset)) != -1; ) {
if (n > 0) { if (n > 0) {
n += offset; n += offset;
@ -801,12 +800,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
} }
} }
} }
}
// calculate and write the last crc // calculate and write the last crc
checksum.calculateChunkedSums(data, 0, offset, crcs, 0); checksum.calculateChunkedSums(data, 0, offset, crcs, 0);
metaOut.write(crcs, 0, 4); metaOut.write(crcs, 0, 4);
} finally { } finally {
IOUtils.cleanup(LOG, dataIn, metaOut); IOUtils.cleanup(LOG, metaOut);
} }
} }
@ -1599,11 +1599,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
} }
f = info.getBlockFile(); f = info.getBlockFile();
v = (FsVolumeImpl)info.getVolume(); v = (FsVolumeImpl)info.getVolume();
if (f == null) {
errors.add("Failed to delete replica " + invalidBlks[i]
+ ": File not found, volume=" + v);
continue;
}
if (v == null) { if (v == null) {
errors.add("Failed to delete replica " + invalidBlks[i] errors.add("Failed to delete replica " + invalidBlks[i]
+ ". No volume for this replica, file=" + f); + ". No volume for this replica, file=" + f);

View File

@ -17,6 +17,7 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode.web.webhdfs; package org.apache.hadoop.hdfs.server.datanode.web.webhdfs;
import com.google.common.base.Charsets;
import com.sun.jersey.api.ParamException; import com.sun.jersey.api.ParamException;
import com.sun.jersey.api.container.ContainerException; import com.sun.jersey.api.container.ContainerException;
import io.netty.buffer.Unpooled; import io.netty.buffer.Unpooled;
@ -39,7 +40,7 @@ import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.APPLICATION_JSON; import static org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.APPLICATION_JSON_UTF8;
class ExceptionHandler { class ExceptionHandler {
static Log LOG = WebHdfsHandler.LOG; static Log LOG = WebHdfsHandler.LOG;
@ -82,11 +83,11 @@ class ExceptionHandler {
s = INTERNAL_SERVER_ERROR; s = INTERNAL_SERVER_ERROR;
} }
final byte[] js = JsonUtil.toJsonString(e).getBytes(); final byte[] js = JsonUtil.toJsonString(e).getBytes(Charsets.UTF_8);
DefaultFullHttpResponse resp = DefaultFullHttpResponse resp =
new DefaultFullHttpResponse(HTTP_1_1, s, Unpooled.wrappedBuffer(js)); new DefaultFullHttpResponse(HTTP_1_1, s, Unpooled.wrappedBuffer(js));
resp.headers().set(CONTENT_TYPE, APPLICATION_JSON); resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
resp.headers().set(CONTENT_LENGTH, js.length); resp.headers().set(CONTENT_LENGTH, js.length);
return resp; return resp;
} }

View File

@ -29,6 +29,7 @@ import io.netty.handler.codec.http.HttpMethod;
import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.QueryStringDecoder; import io.netty.handler.codec.http.QueryStringDecoder;
import io.netty.handler.stream.ChunkedStream; import io.netty.handler.stream.ChunkedStream;
import org.apache.commons.io.Charsets;
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.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -77,7 +78,8 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
public static final int WEBHDFS_PREFIX_LENGTH = WEBHDFS_PREFIX.length(); public static final int WEBHDFS_PREFIX_LENGTH = WEBHDFS_PREFIX.length();
public static final String APPLICATION_OCTET_STREAM = public static final String APPLICATION_OCTET_STREAM =
"application/octet-stream"; "application/octet-stream";
public static final String APPLICATION_JSON = "application/json"; public static final String APPLICATION_JSON_UTF8 =
"application/json; charset=utf-8";
private final Configuration conf; private final Configuration conf;
private final Configuration confForCreate; private final Configuration confForCreate;
@ -224,11 +226,11 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
} finally { } finally {
IOUtils.cleanup(LOG, dfsclient); IOUtils.cleanup(LOG, dfsclient);
} }
final byte[] js = JsonUtil.toJsonString(checksum).getBytes(); final byte[] js = JsonUtil.toJsonString(checksum).getBytes(Charsets.UTF_8);
DefaultFullHttpResponse resp = DefaultFullHttpResponse resp =
new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js)); new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js));
resp.headers().set(CONTENT_TYPE, APPLICATION_JSON); resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
resp.headers().set(CONTENT_LENGTH, js.length); resp.headers().set(CONTENT_LENGTH, js.length);
resp.headers().set(CONNECTION, CLOSE); resp.headers().set(CONNECTION, CLOSE);
ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE); ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);

View File

@ -48,8 +48,10 @@ import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.FileInputStream;
import java.io.FileReader; import java.io.FileReader;
import java.io.IOException; import java.io.IOException;
import java.io.InputStreamReader;
import java.net.URI; import java.net.URI;
import java.text.DateFormat; import java.text.DateFormat;
import java.util.*; import java.util.*;
@ -579,7 +581,8 @@ public class Mover {
private static String[] readPathFile(String file) throws IOException { private static String[] readPathFile(String file) throws IOException {
List<String> list = Lists.newArrayList(); List<String> list = Lists.newArrayList();
BufferedReader reader = new BufferedReader(new FileReader(file)); BufferedReader reader = new BufferedReader(
new InputStreamReader(new FileInputStream(file), "UTF-8"));
try { try {
String line; String line;
while ((line = reader.readLine()) != null) { while ((line = reader.readLine()) != null) {

View File

@ -433,7 +433,6 @@ class FSDirRenameOp {
} else { } else {
fsd.addLastINodeNoQuotaCheck(dstIIP, removedDst); fsd.addLastINodeNoQuotaCheck(dstIIP, removedDst);
} }
assert removedDst != null;
if (removedDst.isReference()) { if (removedDst.isReference()) {
final INodeReference removedDstRef = removedDst.asReference(); final INodeReference removedDstRef = removedDst.asReference();
final INodeReference.WithCount wc = (INodeReference.WithCount) final INodeReference.WithCount wc = (INodeReference.WithCount)

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.commons.io.Charsets;
import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException; import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.FileEncryptionInfo;
@ -50,7 +51,7 @@ class FSDirStatAndListingOp {
FSPermissionChecker pc = fsd.getPermissionChecker(); FSPermissionChecker pc = fsd.getPermissionChecker();
byte[][] pathComponents = FSDirectory byte[][] pathComponents = FSDirectory
.getPathComponentsForReservedPath(srcArg); .getPathComponentsForReservedPath(srcArg);
final String startAfterString = new String(startAfter); final String startAfterString = new String(startAfter, Charsets.UTF_8);
final String src = fsd.resolvePath(pc, srcArg, pathComponents); final String src = fsd.resolvePath(pc, srcArg, pathComponents);
final INodesInPath iip = fsd.getINodesInPath(src, true); final INodesInPath iip = fsd.getINodesInPath(src, true);
@ -195,8 +196,7 @@ class FSDirStatAndListingOp {
cur.getLocalStoragePolicyID(): cur.getLocalStoragePolicyID():
BlockStoragePolicySuite.ID_UNSPECIFIED; BlockStoragePolicySuite.ID_UNSPECIFIED;
listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), cur, listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), cur,
needLocation, fsd.getStoragePolicyID(curPolicy, needLocation, fsd.getStoragePolicyID(curPolicy, parentStoragePolicy), snapshot, isRawPath, inodesInPath);
parentStoragePolicy), snapshot, isRawPath, inodesInPath);
listingCnt++; listingCnt++;
if (needLocation) { if (needLocation) {
// Once we hit lsLimit locations, stop. // Once we hit lsLimit locations, stop.

View File

@ -23,6 +23,7 @@ import java.io.InputStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.util.Arrays; import java.util.Arrays;
import org.apache.commons.io.Charsets;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@ -32,7 +33,8 @@ import org.apache.hadoop.io.compress.CompressionCodec;
@InterfaceAudience.Private @InterfaceAudience.Private
public final class FSImageUtil { public final class FSImageUtil {
public static final byte[] MAGIC_HEADER = "HDFSIMG1".getBytes(); public static final byte[] MAGIC_HEADER =
"HDFSIMG1".getBytes(Charsets.UTF_8);
public static final int FILE_VERSION = 1; public static final int FILE_VERSION = 1;
public static boolean checkFileFormat(RandomAccessFile file) public static boolean checkFileFormat(RandomAccessFile file)

View File

@ -300,7 +300,7 @@ public class FileJournalManager implements JournalManager {
.matcher(name); .matcher(name);
if (staleInprogressEditsMatch.matches()) { if (staleInprogressEditsMatch.matches()) {
try { try {
long startTxId = Long.valueOf(staleInprogressEditsMatch.group(1)); long startTxId = Long.parseLong(staleInprogressEditsMatch.group(1));
ret.add(new EditLogFile(f, startTxId, HdfsConstants.INVALID_TXID, ret.add(new EditLogFile(f, startTxId, HdfsConstants.INVALID_TXID,
true)); true));
continue; continue;

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.PrintStream; import java.io.PrintStream;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.StringWriter; import java.io.StringWriter;
import java.nio.charset.Charset;
import java.util.List; import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -769,8 +770,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
@VisibleForTesting @VisibleForTesting
public final void dumpTreeRecursively(PrintStream out) { public final void dumpTreeRecursively(PrintStream out) {
dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(), out.println(dumpTreeRecursively().toString());
Snapshot.CURRENT_STATE_ID);
} }
/** /**

View File

@ -974,10 +974,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
public DatanodeInfo[] getDatanodeReport(DatanodeReportType type) public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
throws IOException { throws IOException {
DatanodeInfo results[] = namesystem.datanodeReport(type); DatanodeInfo results[] = namesystem.datanodeReport(type);
if (results == null ) {
throw new IOException("Failed to get datanode report for " + type
+ " datanodes.");
}
return results; return results;
} }
@ -985,10 +981,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
public DatanodeStorageReport[] getDatanodeStorageReport( public DatanodeStorageReport[] getDatanodeStorageReport(
DatanodeReportType type) throws IOException { DatanodeReportType type) throws IOException {
final DatanodeStorageReport[] reports = namesystem.getDatanodeStorageReport(type); final DatanodeStorageReport[] reports = namesystem.getDatanodeStorageReport(type);
if (reports == null ) {
throw new IOException("Failed to get datanode storage report for " + type
+ " datanodes.");
}
return reports; return reports;
} }

View File

@ -643,10 +643,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
} }
if (fos == null) { if (fos == null) {
fos = dfs.create(target + "/" + chain, true); fos = dfs.create(target + "/" + chain, true);
if (fos == null) {
throw new IOException("Failed to copy " + fullName +
" to /lost+found: could not store chain " + chain);
}
chain++; chain++;
} }

View File

@ -100,7 +100,7 @@ public class XAttrPermissionFilter {
static List<XAttr> filterXAttrsForApi(FSPermissionChecker pc, static List<XAttr> filterXAttrsForApi(FSPermissionChecker pc,
List<XAttr> xAttrs, boolean isRawPath) { List<XAttr> xAttrs, boolean isRawPath) {
assert xAttrs != null : "xAttrs can not be null"; assert xAttrs != null : "xAttrs can not be null";
if (xAttrs == null || xAttrs.isEmpty()) { if (xAttrs.isEmpty()) {
return xAttrs; return xAttrs;
} }

View File

@ -1476,7 +1476,7 @@ public class DFSAdmin extends FsShell {
} else { } else {
out.print("FAILED: "); out.print("FAILED: ");
} }
out.printf("Change property %s\n\tFrom: \"%s\"\n\tTo: \"%s\"\n", out.printf("Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
result.getKey().prop, result.getKey().oldVal, result.getKey().prop, result.getKey().oldVal,
result.getKey().newVal); result.getKey().newVal);
if (result.getValue().isPresent()) { if (result.getValue().isPresent()) {

View File

@ -144,7 +144,7 @@ class DelimitedImageVisitor extends TextWriterImageVisitor {
// Special case of file size, which is sum of the num bytes in each block // Special case of file size, which is sum of the num bytes in each block
if(element == ImageElement.NUM_BYTES) if(element == ImageElement.NUM_BYTES)
fileSize += Long.valueOf(value); fileSize += Long.parseLong(value);
if(elements.containsKey(element) && element != ImageElement.NUM_BYTES) if(elements.containsKey(element) && element != ImageElement.NUM_BYTES)
elements.put(element, value); elements.put(element, value);

View File

@ -17,11 +17,7 @@
*/ */
package org.apache.hadoop.hdfs.tools.offlineImageViewer; package org.apache.hadoop.hdfs.tools.offlineImageViewer;
import java.io.FileNotFoundException; import com.google.common.base.Charsets;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled; import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelFutureListener;
@ -30,19 +26,31 @@ import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.group.ChannelGroup; import io.netty.channel.group.ChannelGroup;
import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.DefaultHttpResponse; import io.netty.handler.codec.http.DefaultHttpResponse;
import static io.netty.handler.codec.http.HttpResponseStatus.*;
import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpMethod;
import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpResponseStatus;
import static io.netty.handler.codec.http.HttpVersion.*;
import io.netty.handler.codec.http.QueryStringDecoder; import io.netty.handler.codec.http.QueryStringDecoder;
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.hdfs.web.JsonUtil; import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
import static io.netty.handler.codec.http.HttpHeaders.Values.CLOSE;
import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.APPLICATION_JSON_UTF8;
import static org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.WEBHDFS_PREFIX;
import static org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.WEBHDFS_PREFIX_LENGTH;
/** /**
* Implement the read-only WebHDFS API for fsimage. * Implement the read-only WebHDFS API for fsimage.
*/ */
@ -67,7 +75,7 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
if (request.getMethod() != HttpMethod.GET) { if (request.getMethod() != HttpMethod.GET) {
DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1,
METHOD_NOT_ALLOWED); METHOD_NOT_ALLOWED);
resp.headers().set("Connection", "close"); resp.headers().set(CONNECTION, CLOSE);
ctx.write(resp).addListener(ChannelFutureListener.CLOSE); ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
return; return;
} }
@ -77,24 +85,29 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
final String content; final String content;
String path = getPath(decoder); String path = getPath(decoder);
if ("GETFILESTATUS".equals(op)) { switch (op) {
case "GETFILESTATUS":
content = image.getFileStatus(path); content = image.getFileStatus(path);
} else if ("LISTSTATUS".equals(op)) { break;
case "LISTSTATUS":
content = image.listStatus(path); content = image.listStatus(path);
} else if ("GETACLSTATUS".equals(op)) { break;
case "GETACLSTATUS":
content = image.getAclStatus(path); content = image.getAclStatus(path);
} else { break;
throw new IllegalArgumentException("Invalid value for webhdfs parameter" + " \"op\""); default:
throw new IllegalArgumentException(
"Invalid value for webhdfs parameter" + " \"op\"");
} }
LOG.info("op=" + op + " target=" + path); LOG.info("op=" + op + " target=" + path);
DefaultFullHttpResponse resp = new DefaultFullHttpResponse( DefaultFullHttpResponse resp = new DefaultFullHttpResponse(
HTTP_1_1, HttpResponseStatus.OK, HTTP_1_1, HttpResponseStatus.OK,
Unpooled.wrappedBuffer(content.getBytes())); Unpooled.wrappedBuffer(content.getBytes(Charsets.UTF_8)));
resp.headers().set("Content-Type", "application/json"); resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
resp.headers().set("Content-Length", resp.content().readableBytes()); resp.headers().set(CONTENT_LENGTH, resp.content().readableBytes());
resp.headers().set("Connection", "close"); resp.headers().set(CONNECTION, CLOSE);
ctx.write(resp).addListener(ChannelFutureListener.CLOSE); ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
} }
@ -109,19 +122,19 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
Exception e = cause instanceof Exception ? (Exception) cause : new Exception e = cause instanceof Exception ? (Exception) cause : new
Exception(cause); Exception(cause);
final String output = JsonUtil.toJsonString(e); final String output = JsonUtil.toJsonString(e);
ByteBuf content = Unpooled.wrappedBuffer(output.getBytes()); ByteBuf content = Unpooled.wrappedBuffer(output.getBytes(Charsets.UTF_8));
final DefaultFullHttpResponse resp = new DefaultFullHttpResponse( final DefaultFullHttpResponse resp = new DefaultFullHttpResponse(
HTTP_1_1, INTERNAL_SERVER_ERROR, content); HTTP_1_1, INTERNAL_SERVER_ERROR, content);
resp.headers().set("Content-Type", "application/json"); resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
if (e instanceof IllegalArgumentException) { if (e instanceof IllegalArgumentException) {
resp.setStatus(BAD_REQUEST); resp.setStatus(BAD_REQUEST);
} else if (e instanceof FileNotFoundException) { } else if (e instanceof FileNotFoundException) {
resp.setStatus(NOT_FOUND); resp.setStatus(NOT_FOUND);
} }
resp.headers().set("Content-Length", resp.content().readableBytes()); resp.headers().set(CONTENT_LENGTH, resp.content().readableBytes());
resp.headers().set("Connection", "close"); resp.headers().set(CONNECTION, CLOSE);
ctx.write(resp).addListener(ChannelFutureListener.CLOSE); ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
} }
@ -134,11 +147,11 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
private static String getPath(QueryStringDecoder decoder) private static String getPath(QueryStringDecoder decoder)
throws FileNotFoundException { throws FileNotFoundException {
String path = decoder.path(); String path = decoder.path();
if (path.startsWith("/webhdfs/v1/")) { if (path.startsWith(WEBHDFS_PREFIX)) {
return path.substring(11); return path.substring(WEBHDFS_PREFIX_LENGTH);
} else { } else {
throw new FileNotFoundException("Path: " + path + " should " + throw new FileNotFoundException("Path: " + path + " should " +
"start with \"/webhdfs/v1/\""); "start with " + WEBHDFS_PREFIX);
} }
} }
} }

View File

@ -111,17 +111,15 @@ class FSImageLoader {
} }
FsImageProto.FileSummary summary = FSImageUtil.loadSummary(file); FsImageProto.FileSummary summary = FSImageUtil.loadSummary(file);
FileInputStream fin = null;
try {
try (FileInputStream fin = new FileInputStream(file.getFD())) {
// Map to record INodeReference to the referred id // Map to record INodeReference to the referred id
ImmutableList<Long> refIdList = null; ImmutableList<Long> refIdList = null;
String[] stringTable = null; String[] stringTable = null;
byte[][] inodes = null; byte[][] inodes = null;
Map<Long, long[]> dirmap = null; Map<Long, long[]> dirmap = null;
fin = new FileInputStream(file.getFD());
ArrayList<FsImageProto.FileSummary.Section> sections = ArrayList<FsImageProto.FileSummary.Section> sections =
Lists.newArrayList(summary.getSectionsList()); Lists.newArrayList(summary.getSectionsList());
Collections.sort(sections, Collections.sort(sections,
@ -169,8 +167,6 @@ class FSImageLoader {
} }
} }
return new FSImageLoader(stringTable, inodes, dirmap); return new FSImageLoader(stringTable, inodes, dirmap);
} finally {
IOUtils.cleanup(null, fin);
} }
} }

View File

@ -21,7 +21,7 @@ import java.io.BufferedInputStream;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.PrintWriter; import java.io.PrintStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
import org.apache.hadoop.hdfs.server.namenode.FSImageUtil; import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.LimitInputStream; import org.apache.hadoop.util.LimitInputStream;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -67,7 +66,7 @@ final class FileDistributionCalculator {
private final Configuration conf; private final Configuration conf;
private final long maxSize; private final long maxSize;
private final int steps; private final int steps;
private final PrintWriter out; private final PrintStream out;
private final int[] distribution; private final int[] distribution;
private int totalFiles; private int totalFiles;
@ -77,7 +76,7 @@ final class FileDistributionCalculator {
private long maxFileSize; private long maxFileSize;
FileDistributionCalculator(Configuration conf, long maxSize, int steps, FileDistributionCalculator(Configuration conf, long maxSize, int steps,
PrintWriter out) { PrintStream out) {
this.conf = conf; this.conf = conf;
this.maxSize = maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize; this.maxSize = maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize;
this.steps = steps == 0 ? INTERVAL_DEFAULT : steps; this.steps = steps == 0 ? INTERVAL_DEFAULT : steps;
@ -96,9 +95,7 @@ final class FileDistributionCalculator {
} }
FileSummary summary = FSImageUtil.loadSummary(file); FileSummary summary = FSImageUtil.loadSummary(file);
FileInputStream in = null; try (FileInputStream in = new FileInputStream(file.getFD())) {
try {
in = new FileInputStream(file.getFD());
for (FileSummary.Section s : summary.getSectionsList()) { for (FileSummary.Section s : summary.getSectionsList()) {
if (SectionName.fromString(s.getName()) != SectionName.INODE) { if (SectionName.fromString(s.getName()) != SectionName.INODE) {
continue; continue;
@ -111,8 +108,6 @@ final class FileDistributionCalculator {
run(is); run(is);
output(); output();
} }
} finally {
IOUtils.cleanup(null, in);
} }
} }

View File

@ -159,10 +159,10 @@ class FileDistributionVisitor extends TextWriterImageVisitor {
current.path = (value.equals("") ? "/" : value); current.path = (value.equals("") ? "/" : value);
break; break;
case REPLICATION: case REPLICATION:
current.replication = Integer.valueOf(value); current.replication = Integer.parseInt(value);
break; break;
case NUM_BYTES: case NUM_BYTES:
current.fileSize += Long.valueOf(value); current.fileSize += Long.parseLong(value);
break; break;
default: default:
break; break;

View File

@ -135,7 +135,7 @@ class LsImageVisitor extends TextWriterImageVisitor {
perms = value; perms = value;
break; break;
case REPLICATION: case REPLICATION:
replication = Integer.valueOf(value); replication = Integer.parseInt(value);
break; break;
case USER_NAME: case USER_NAME:
username = value; username = value;
@ -144,7 +144,7 @@ class LsImageVisitor extends TextWriterImageVisitor {
group = value; group = value;
break; break;
case NUM_BYTES: case NUM_BYTES:
filesize += Long.valueOf(value); filesize += Long.parseLong(value);
break; break;
case MODIFICATION_TIME: case MODIFICATION_TIME:
modTime = value; modTime = value;
@ -173,6 +173,6 @@ class LsImageVisitor extends TextWriterImageVisitor {
if(element == ImageElement.INODE) if(element == ImageElement.INODE)
newLine(); newLine();
else if (element == ImageElement.BLOCKS) else if (element == ImageElement.BLOCKS)
numBlocks = Integer.valueOf(value); numBlocks = Integer.parseInt(value);
} }
} }

View File

@ -18,9 +18,8 @@
package org.apache.hadoop.hdfs.tools.offlineImageViewer; package org.apache.hadoop.hdfs.tools.offlineImageViewer;
import java.io.EOFException; import java.io.EOFException;
import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
@ -33,7 +32,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
/** /**
@ -144,36 +142,33 @@ public class OfflineImageViewerPB {
String processor = cmd.getOptionValue("p", "Web"); String processor = cmd.getOptionValue("p", "Web");
String outputFile = cmd.getOptionValue("o", "-"); String outputFile = cmd.getOptionValue("o", "-");
PrintWriter out = outputFile.equals("-") ?
new PrintWriter(System.out) : new PrintWriter(new File(outputFile));
Configuration conf = new Configuration(); Configuration conf = new Configuration();
try { try (PrintStream out = outputFile.equals("-") ?
if (processor.equals("FileDistribution")) { System.out : new PrintStream(outputFile, "UTF-8")) {
switch (processor) {
case "FileDistribution":
long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0")); long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0"));
int step = Integer.parseInt(cmd.getOptionValue("step", "0")); int step = Integer.parseInt(cmd.getOptionValue("step", "0"));
new FileDistributionCalculator(conf, maxSize, step, out) new FileDistributionCalculator(conf, maxSize, step, out).visit(
.visit(new RandomAccessFile(inputFile, "r")); new RandomAccessFile(inputFile, "r"));
} else if (processor.equals("XML")) { break;
new PBImageXmlWriter(conf, out).visit(new RandomAccessFile(inputFile, case "XML":
"r")); new PBImageXmlWriter(conf, out).visit(
} else if (processor.equals("Web")) { new RandomAccessFile(inputFile, "r"));
break;
case "Web":
String addr = cmd.getOptionValue("addr", "localhost:5978"); String addr = cmd.getOptionValue("addr", "localhost:5978");
WebImageViewer viewer = new WebImageViewer(NetUtils.createSocketAddr try (WebImageViewer viewer = new WebImageViewer(
(addr)); NetUtils.createSocketAddr(addr))) {
try {
viewer.start(inputFile); viewer.start(inputFile);
} finally {
viewer.close();
} }
break;
} }
return 0; return 0;
} catch (EOFException e) { } catch (EOFException e) {
System.err.println("Input file ended unexpectedly. Exiting"); System.err.println("Input file ended unexpectedly. Exiting");
} catch (IOException e) { } catch (IOException e) {
System.err.println("Encountered exception. Exiting: " + e.getMessage()); System.err.println("Encountered exception. Exiting: " + e.getMessage());
} finally {
IOUtils.cleanup(null, out);
} }
return -1; return -1;
} }

View File

@ -21,7 +21,7 @@ import java.io.BufferedInputStream;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.PrintWriter; import java.io.PrintStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -50,7 +50,6 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
import org.apache.hadoop.hdfs.util.XMLUtils; import org.apache.hadoop.hdfs.util.XMLUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.LimitInputStream; import org.apache.hadoop.util.LimitInputStream;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -62,10 +61,10 @@ import com.google.common.collect.Lists;
@InterfaceAudience.Private @InterfaceAudience.Private
public final class PBImageXmlWriter { public final class PBImageXmlWriter {
private final Configuration conf; private final Configuration conf;
private final PrintWriter out; private final PrintStream out;
private String[] stringTable; private String[] stringTable;
public PBImageXmlWriter(Configuration conf, PrintWriter out) { public PBImageXmlWriter(Configuration conf, PrintStream out) {
this.conf = conf; this.conf = conf;
this.out = out; this.out = out;
} }
@ -76,9 +75,7 @@ public final class PBImageXmlWriter {
} }
FileSummary summary = FSImageUtil.loadSummary(file); FileSummary summary = FSImageUtil.loadSummary(file);
FileInputStream fin = null; try (FileInputStream fin = new FileInputStream(file.getFD())) {
try {
fin = new FileInputStream(file.getFD());
out.print("<?xml version=\"1.0\"?>\n<fsimage>"); out.print("<?xml version=\"1.0\"?>\n<fsimage>");
ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
@ -140,8 +137,6 @@ public final class PBImageXmlWriter {
} }
} }
out.print("</fsimage>\n"); out.print("</fsimage>\n");
} finally {
IOUtils.cleanup(null, fin);
} }
} }

View File

@ -1265,11 +1265,11 @@ public class TestEncryptionZones {
} }
// Run the XML OIV processor // Run the XML OIV processor
StringWriter output = new StringWriter(); ByteArrayOutputStream output = new ByteArrayOutputStream();
PrintWriter pw = new PrintWriter(output); PrintStream pw = new PrintStream(output);
PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), pw); PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), pw);
v.visit(new RandomAccessFile(originalFsimage, "r")); v.visit(new RandomAccessFile(originalFsimage, "r"));
final String xml = output.getBuffer().toString(); final String xml = output.toString();
SAXParser parser = SAXParserFactory.newInstance().newSAXParser(); SAXParser parser = SAXParserFactory.newInstance().newSAXParser();
parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler()); parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
} }

View File

@ -25,15 +25,15 @@ import static org.junit.Assert.fail;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.io.StringWriter;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.Random; import java.util.Random;
import org.apache.commons.io.output.NullOutputStream;
import org.apache.commons.logging.impl.Log4JLogger; import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree.Node; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree.Node;
import org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter; import org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -256,8 +257,7 @@ public class TestSnapshot {
FSImageTestUtil.getFSImage( FSImageTestUtil.getFSImage(
cluster.getNameNode()).getStorage().getStorageDir(0)); cluster.getNameNode()).getStorage().getStorageDir(0));
assertNotNull("Didn't generate or can't find fsimage", originalFsimage); assertNotNull("Didn't generate or can't find fsimage", originalFsimage);
StringWriter output = new StringWriter(); PrintStream o = new PrintStream(NullOutputStream.NULL_OUTPUT_STREAM);
PrintWriter o = new PrintWriter(output);
PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o); PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o);
v.visit(new RandomAccessFile(originalFsimage, "r")); v.visit(new RandomAccessFile(originalFsimage, "r"));
} }

View File

@ -20,18 +20,18 @@ package org.apache.hadoop.hdfs.tools.offlineImageViewer;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.File; import java.io.File;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.io.StringReader; import java.io.StringReader;
import java.io.StringWriter; import java.io.StringWriter;
import java.net.HttpURLConnection; import java.net.HttpURLConnection;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL; import java.net.URL;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
@ -43,6 +43,7 @@ import javax.xml.parsers.ParserConfigurationException;
import javax.xml.parsers.SAXParser; import javax.xml.parsers.SAXParser;
import javax.xml.parsers.SAXParserFactory; import javax.xml.parsers.SAXParserFactory;
import org.apache.commons.io.output.NullOutputStream;
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.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -186,10 +187,10 @@ public class TestOfflineImageViewer {
@Test(expected = IOException.class) @Test(expected = IOException.class)
public void testTruncatedFSImage() throws IOException { public void testTruncatedFSImage() throws IOException {
File truncatedFile = folder.newFile(); File truncatedFile = folder.newFile();
StringWriter output = new StringWriter(); PrintStream output = new PrintStream(NullOutputStream.NULL_OUTPUT_STREAM);
copyPartOfFile(originalFsimage, truncatedFile); copyPartOfFile(originalFsimage, truncatedFile);
new FileDistributionCalculator(new Configuration(), 0, 0, new PrintWriter( new FileDistributionCalculator(new Configuration(), 0, 0, output)
output)).visit(new RandomAccessFile(truncatedFile, "r")); .visit(new RandomAccessFile(truncatedFile, "r"));
} }
private void copyPartOfFile(File src, File dest) throws IOException { private void copyPartOfFile(File src, File dest) throws IOException {
@ -208,20 +209,21 @@ public class TestOfflineImageViewer {
@Test @Test
public void testFileDistributionCalculator() throws IOException { public void testFileDistributionCalculator() throws IOException {
StringWriter output = new StringWriter(); ByteArrayOutputStream output = new ByteArrayOutputStream();
PrintWriter o = new PrintWriter(output); PrintStream o = new PrintStream(output);
new FileDistributionCalculator(new Configuration(), 0, 0, o) new FileDistributionCalculator(new Configuration(), 0, 0, o)
.visit(new RandomAccessFile(originalFsimage, "r")); .visit(new RandomAccessFile(originalFsimage, "r"));
o.close(); o.close();
String outputString = output.toString();
Pattern p = Pattern.compile("totalFiles = (\\d+)\n"); Pattern p = Pattern.compile("totalFiles = (\\d+)\n");
Matcher matcher = p.matcher(output.getBuffer()); Matcher matcher = p.matcher(outputString);
assertTrue(matcher.find() && matcher.groupCount() == 1); assertTrue(matcher.find() && matcher.groupCount() == 1);
int totalFiles = Integer.parseInt(matcher.group(1)); int totalFiles = Integer.parseInt(matcher.group(1));
assertEquals(NUM_DIRS * FILES_PER_DIR, totalFiles); assertEquals(NUM_DIRS * FILES_PER_DIR, totalFiles);
p = Pattern.compile("totalDirectories = (\\d+)\n"); p = Pattern.compile("totalDirectories = (\\d+)\n");
matcher = p.matcher(output.getBuffer()); matcher = p.matcher(outputString);
assertTrue(matcher.find() && matcher.groupCount() == 1); assertTrue(matcher.find() && matcher.groupCount() == 1);
int totalDirs = Integer.parseInt(matcher.group(1)); int totalDirs = Integer.parseInt(matcher.group(1));
// totalDirs includes root directory, empty directory, and xattr directory // totalDirs includes root directory, empty directory, and xattr directory
@ -236,7 +238,7 @@ public class TestOfflineImageViewer {
} }
}); });
p = Pattern.compile("maxFileSize = (\\d+)\n"); p = Pattern.compile("maxFileSize = (\\d+)\n");
matcher = p.matcher(output.getBuffer()); matcher = p.matcher(output.toString("UTF-8"));
assertTrue(matcher.find() && matcher.groupCount() == 1); assertTrue(matcher.find() && matcher.groupCount() == 1);
assertEquals(maxFile.getLen(), Long.parseLong(matcher.group(1))); assertEquals(maxFile.getLen(), Long.parseLong(matcher.group(1)));
} }
@ -252,13 +254,13 @@ public class TestOfflineImageViewer {
@Test @Test
public void testPBImageXmlWriter() throws IOException, SAXException, public void testPBImageXmlWriter() throws IOException, SAXException,
ParserConfigurationException { ParserConfigurationException {
StringWriter output = new StringWriter(); ByteArrayOutputStream output = new ByteArrayOutputStream();
PrintWriter o = new PrintWriter(output); PrintStream o = new PrintStream(output);
PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o); PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o);
v.visit(new RandomAccessFile(originalFsimage, "r")); v.visit(new RandomAccessFile(originalFsimage, "r"));
SAXParserFactory spf = SAXParserFactory.newInstance(); SAXParserFactory spf = SAXParserFactory.newInstance();
SAXParser parser = spf.newSAXParser(); SAXParser parser = spf.newSAXParser();
final String xml = output.getBuffer().toString(); final String xml = output.toString();
parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler()); parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
} }
@ -298,7 +300,7 @@ public class TestOfflineImageViewer {
verifyHttpResponseCode(HttpURLConnection.HTTP_NOT_FOUND, url); verifyHttpResponseCode(HttpURLConnection.HTTP_NOT_FOUND, url);
// LISTSTATUS operation to a invalid prefix // LISTSTATUS operation to a invalid prefix
url = new URL("http://localhost:" + port + "/webhdfs/v1?op=LISTSTATUS"); url = new URL("http://localhost:" + port + "/foo");
verifyHttpResponseCode(HttpURLConnection.HTTP_NOT_FOUND, url); verifyHttpResponseCode(HttpURLConnection.HTTP_NOT_FOUND, url);
// GETFILESTATUS operation // GETFILESTATUS operation