HDFS-7515. Fix new findbugs warnings in hadoop-hdfs. Contributed by Haohui Mai.
This commit is contained in:
parent
614b6afea4
commit
b9f6d0c956
|
@ -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
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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++;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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()) {
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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());
|
||||||
}
|
}
|
||||||
|
|
|
@ -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"));
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
Loading…
Reference in New Issue