svn merge -c 1574259 from trunk for HDFS-5535.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1576128 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2014-03-10 23:37:46 +00:00
parent 0663d54767
commit 134a29f226
145 changed files with 6524 additions and 1288 deletions

View File

@ -244,13 +244,14 @@ public void printDetails(final PrintStream out) {
out.println("\n]"); out.println("\n]");
} }
private class SetIterator implements Iterator<E> { public class SetIterator implements Iterator<E> {
/** The starting modification for fail-fast. */ /** The starting modification for fail-fast. */
private int iterModification = modification; private int iterModification = modification;
/** The current index of the entry array. */ /** The current index of the entry array. */
private int index = -1; private int index = -1;
private LinkedElement cur = null; private LinkedElement cur = null;
private LinkedElement next = nextNonemptyEntry(); private LinkedElement next = nextNonemptyEntry();
private boolean trackModification = true;
/** Find the next nonempty entry starting at (index + 1). */ /** Find the next nonempty entry starting at (index + 1). */
private LinkedElement nextNonemptyEntry() { private LinkedElement nextNonemptyEntry() {
@ -259,7 +260,7 @@ private LinkedElement nextNonemptyEntry() {
} }
private void ensureNext() { private void ensureNext() {
if (modification != iterModification) { if (trackModification && modification != iterModification) {
throw new ConcurrentModificationException("modification=" + modification throw new ConcurrentModificationException("modification=" + modification
+ " != iterModification = " + iterModification); + " != iterModification = " + iterModification);
} }
@ -304,6 +305,10 @@ public void remove() {
iterModification++; iterModification++;
cur = null; cur = null;
} }
public void setTrackModification(boolean trackModification) {
this.trackModification = trackModification;
}
} }
/** /**

View File

@ -501,6 +501,148 @@ Release 2.4.0 - UNRELEASED
HDFS-6069. Quash stack traces when ACLs are disabled. (cnauroth) HDFS-6069. Quash stack traces when ACLs are disabled. (cnauroth)
HDFS-5535 subtasks:
HDFS-5496. Make replication queue initialization asynchronous. (Vinay via
jing9)
HDFS-5645. Support upgrade marker in editlog streams. (szetszwo)
HDFS-5752. Add a new DFSAdmin command to query, start and finalize rolling
upgrade. (szetszwo)
HDFS-5786. Support QUERY and FINALIZE actions of rolling upgrade. (szetszwo)
HDFS-5753. Add new Namenode startup options for downgrade and rollback using
upgrade marker. (szetszwo)
HDFS-5835. Add a new option for starting Namenode when rolling upgrade is
in progress. (szetszwo)
HDFS-5754. Split LayoutVerion into NameNodeLayoutVersion and
DataNodeLayoutVersion. (Brandon Li via szetszwo)
HDFS-5848. Add rolling upgrade status to heartbeat response. (szetszwo)
HDFS-5890. Avoid NPE in Datanode heartbeat. (Vinay via brandonli)
HDFS-5869. When starting rolling upgrade or NN restarts, NN should create
a checkpoint right before the upgrade marker. (szetszwo)
HDFS-5874. Should not compare DataNode current layout version with that of
NameNode in DataStrorage. (brandonli)
HDFS-5889. When starting rolling upgrade, create a fs image for rollback
so that the standby namenode can create checkpoints during upgrade.
(szetszwo & jing9)
HDFS-5907. Add BlockPoolSliceStorage 'trash' to handle block deletions
during rolling upgrades. (Arpit Agarwal)
HDFS-5494. Merge Protobuf-based-FSImage code from trunk - fix build
break after the merge. (Jing Zhao via Arpit Agarwal)
HDFS-5585. Provide admin commands for data node upgrade (kihwal)
HDFS-5920. Support rollback of rolling upgrade in NameNode and JournalNodes.
(jing9)
HDFS-5945. Add rolling upgrade information to fsimage; and disallow upgrade
and rolling upgrade to be started simultaneously. (szetszwo & jing9)
HDFS-5966. Fix rollback of rolling upgrade in NameNode HA setup. (jing9
via szetszwo)
HDFS-5974. Fix compilation error, NameNodeLayoutVersion and
DataNodeLayoutVersion after merge from trunk. (szetszwo)
HDFS-5963. TestRollingUpgrade#testSecondaryNameNode causes subsequent
tests to fail. (szetszwo via Arpit Agarwal)
HDFS-5976. Create unit tests for downgrade and finalize rolling upgrade.
(Haohui Mai via Arpit Agarwal)
HDFS-5980. Rollback does not need to load edits. (jing9 via szetszwo)
HDFS-5984. Fix TestEditLog and TestStandbyCheckpoints. (jing9 via szetszwo)
HDFS-5985. SimulatedFSDataset#disableAndPurgeTrashStorage should not throw
UnsupportedOperationException. (jing9 via kihwal)
HDFS-5987. Fix findbugs warnings in Rolling Upgrade branch. (seztszwo via
Arpit Agarwal)
HDFS-5992. Fix NPE in MD5FileUtils and update editsStored for
TestOfflineEditsViewer. (szetszwo)
HDFS-5994. Fix TestDataNodeRollingUpgrade. (Arpit Agarwal via szetszwo)
HDFS-5999. Do not create rollback fsimage when it already exists. (jing9)
HDFS-6005. Simplify Datanode rollback and downgrade. (Suresh Srinivas via
Arpit Agarwal)
HDFS-6004. Change DFSAdmin for rolling upgrade commands. (szetszwo via
Arpit Agarwal)
HDFS-5583. Make DN send an OOB Ack on shutdown before restarting. (kihwal)
HDFS-5778. Add rolling upgrade user document. (szetszwo)
HDFS-6003. Add the new -rollingUpgrade startup option to the namenode
usage message. (Vinayakumar B via szetszwo)
HDFS-6014. Fix findbug warnings introduced by HDFS-5583. (kihwal)
HDFS-6015. Fix TestBlockRecovery
#testRaceBetweenReplicaRecoveryAndFinalizeBlock. (kihwal)
HDFS-5924. Utilize OOB upgrade message processing for writes. (kihwal)
HDFS-5498. Improve datanode startup time. (kihwal)
HDFS-6000. Avoid saving namespace when starting rolling upgrade. (jing9)
HDFS-6017. Query the status of rolling upgrade in the preparation stage in
TestRollingUpgrade and TestRollingUpgradeRollback. (Haohui Mai via
Arpit Agarwal)
HDFS-6020. Fix the five findbugs warnings. (kihwal)
HDFS-6019. Standby NN might not checkpoint when processing the rolling
upgrade marker. (Haohui Mai via jing9)
HDFS-6023. Test whether the standby NN continues to checkpoint after the
prepare stage. (Haohui Mai via jing9)
HDFS-6024. Test whether the NN will reject the downgrade if it has a
fsimage from a newer release. (Haohui Mai via jing9)
HDFS-6026. Fix TestDFSUpgrade and TestDataNodeRollingUpgrade.
(jing9 via szetszwo)
HDFS-6029. Secondary NN fails to checkpoint after -rollingUpgrade prepare.
(jing9)
HDFS-6032. -rollingUpgrade query hits NPE after the NN restarts. (Haohui Mai
via jing9)
HDFS-6031. Add back the "-rollingUpgrade started" namenode startup option;
otherwise, namenode cannot start when the layout version is changed.
(szetszwo)
HDFS-6034. Use DataNodeLayoutVersion for DN registration check and do not
verify layout version if there is a rolling upgrade in progress. (szetszwo)
HDFS-6013. add rollingUpgrade information to latest UI.
(Vinayakumar B via wheat9)
HDFS-6042. Fix rolling upgrade documentation and error messages. (szetszwo
via Arpit Agarwal)
HDFS-6041. Downgrade/Finalize should rename the rollback image instead of
purging it. (jing9)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -395,7 +395,7 @@ public EditLogOutputStream startLogSegment(long txId) throws IOException {
try { try {
String znodePath = inprogressZNode(txId); String znodePath = inprogressZNode(txId);
EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath, EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
HdfsConstants.LAYOUT_VERSION, currentLedger.getId(), txId); HdfsConstants.NAMENODE_LAYOUT_VERSION, currentLedger.getId(), txId);
/* Write the ledger metadata out to the inprogress ledger znode /* Write the ledger metadata out to the inprogress ledger znode
* This can fail if for some reason our write lock has * This can fail if for some reason our write lock has
* expired (@see WriteLock) and another process has managed to * expired (@see WriteLock) and another process has managed to
@ -659,6 +659,11 @@ public void purgeLogsOlderThan(long minTxIdToKeep)
} }
} }
@Override
public void discardSegments(long startTxId) throws IOException {
throw new UnsupportedOperationException();
}
@Override @Override
public void close() throws IOException { public void close() throws IOException {
try { try {

View File

@ -68,7 +68,7 @@ public void testEmptyInputStream() throws Exception {
lh.close(); lh.close();
EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar", EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
HdfsConstants.LAYOUT_VERSION, lh.getId(), 0x1234); HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
try { try {
new BookKeeperEditLogInputStream(lh, metadata, -1); new BookKeeperEditLogInputStream(lh, metadata, -1);
fail("Shouldn't get this far, should have thrown"); fail("Shouldn't get this far, should have thrown");
@ -77,7 +77,7 @@ public void testEmptyInputStream() throws Exception {
} }
metadata = new EditLogLedgerMetadata("/foobar", metadata = new EditLogLedgerMetadata("/foobar",
HdfsConstants.LAYOUT_VERSION, lh.getId(), 0x1234); HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
try { try {
new BookKeeperEditLogInputStream(lh, metadata, 0); new BookKeeperEditLogInputStream(lh, metadata, 0);
fail("Shouldn't get this far, should have thrown"); fail("Shouldn't get this far, should have thrown");

View File

@ -30,6 +30,8 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
@ -138,11 +140,13 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -269,6 +273,7 @@ public static class Conf {
final int getFileBlockStorageLocationsTimeoutMs; final int getFileBlockStorageLocationsTimeoutMs;
final int retryTimesForGetLastBlockLength; final int retryTimesForGetLastBlockLength;
final int retryIntervalForGetLastBlockLength; final int retryIntervalForGetLastBlockLength;
final long datanodeRestartTimeout;
final boolean useLegacyBlockReader; final boolean useLegacyBlockReader;
final boolean useLegacyBlockReaderLocal; final boolean useLegacyBlockReaderLocal;
@ -419,6 +424,10 @@ public Conf(Configuration conf) {
shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt( shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT); DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
datanodeRestartTimeout = conf.getLong(
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
} }
private DataChecksum.Type getChecksumType(Configuration conf) { private DataChecksum.Type getChecksumType(Configuration conf) {
@ -2476,6 +2485,10 @@ public void finalizeUpgrade() throws IOException {
namenode.finalizeUpgrade(); namenode.finalizeUpgrade();
} }
RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
return namenode.rollingUpgrade(action);
}
/** /**
*/ */
@Deprecated @Deprecated

View File

@ -94,6 +94,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final long DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT = 3000; public static final long DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT = 3000;
public static final String DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL = "dfs.client.write.exclude.nodes.cache.expiry.interval.millis"; public static final String DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL = "dfs.client.write.exclude.nodes.cache.expiry.interval.millis";
public static final long DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10 * 60 * 1000; // 10 minutes, in ms public static final long DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10 * 60 * 1000; // 10 minutes, in ms
public static final String DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY = "dfs.client.datanode-restart.timeout";
public static final long DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT = 30;
public static final String DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";
public static final long DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT = 50;
public static final String DFS_NAMENODE_BACKUP_ADDRESS_KEY = "dfs.namenode.backup.address"; public static final String DFS_NAMENODE_BACKUP_ADDRESS_KEY = "dfs.namenode.backup.address";
public static final String DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT = "localhost:50100"; public static final String DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT = "localhost:50100";
public static final String DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY = "dfs.namenode.backup.http-address"; public static final String DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY = "dfs.namenode.backup.http-address";
@ -224,6 +228,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_DATANODE_SYNCONCLOSE_DEFAULT = false; public static final boolean DFS_DATANODE_SYNCONCLOSE_DEFAULT = false;
public static final String DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive"; public static final String DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
public static final int DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 4000; public static final int DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 4000;
public static final String DFS_DATANODE_OOB_TIMEOUT_KEY = "dfs.datanode.oob.timeout-ms";
public static final String DFS_DATANODE_OOB_TIMEOUT_DEFAULT = "1500,0,0,0"; // OOB_TYPE1, OOB_TYPE2, OOB_TYPE3, OOB_TYPE4
public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check"; public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check";
public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true; public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true;
@ -416,7 +422,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000; public static final int DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
public static final String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max"; public static final String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
public static final int DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500; public static final int DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500;
/* Maximum number of blocks to process for initializing replication queues */
public static final String DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT = "dfs.block.misreplication.processing.limit";
public static final int DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT = 10000;
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_KEY = "dfs.client.read.shortcircuit"; public static final String DFS_CLIENT_READ_SHORTCIRCUIT_KEY = "dfs.client.read.shortcircuit";
public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT = false; public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT = false;
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY = "dfs.client.read.shortcircuit.skip.checksum"; public static final String DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY = "dfs.client.read.shortcircuit.skip.checksum";

View File

@ -27,6 +27,7 @@
import java.io.InputStream; import java.io.InputStream;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.Socket; import java.net.Socket;
import java.nio.BufferOverflowException; import java.nio.BufferOverflowException;
@ -344,6 +345,8 @@ public DatanodeInfo load(DatanodeInfo key) throws Exception {
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
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
@ -479,7 +482,7 @@ public void run() {
try { try {
// process datanode IO errors if any // process datanode IO errors if any
boolean doSleep = false; boolean doSleep = false;
if (hasError && errorIndex>=0) { if (hasError && (errorIndex >= 0 || restartingNodeIndex >= 0)) {
doSleep = processDatanodeError(); doSleep = processDatanodeError();
} }
@ -577,8 +580,12 @@ public void run() {
blockStream.flush(); blockStream.flush();
} catch (IOException e) { } catch (IOException e) {
// HDFS-3398 treat primary DN is down since client is unable to // HDFS-3398 treat primary DN is down since client is unable to
// write to primary DN // write to primary DN. If a failed or restarting node has already
errorIndex = 0; // been recorded by the responder, the following call will have no
// effect. Pipeline recovery can handle only one node error at a
// time. If the primary node fails again during the recovery, it
// will be taken out then.
tryMarkPrimaryDatanodeFailed();
throw e; throw e;
} }
lastPacket = Time.now(); lastPacket = Time.now();
@ -615,12 +622,16 @@ public void run() {
Thread.sleep(artificialSlowdown); Thread.sleep(artificialSlowdown);
} }
} catch (Throwable e) { } catch (Throwable e) {
DFSClient.LOG.warn("DataStreamer Exception", e); // Log warning if there was a real error.
if (restartingNodeIndex == -1) {
DFSClient.LOG.warn("DataStreamer Exception", e);
}
if (e instanceof IOException) { if (e instanceof IOException) {
setLastException((IOException)e); setLastException((IOException)e);
} }
hasError = true; hasError = true;
if (errorIndex == -1) { // not a datanode error if (errorIndex == -1 && restartingNodeIndex == -1) {
// Not a datanode issue
streamerClosed = true; streamerClosed = true;
} }
} }
@ -699,6 +710,65 @@ private void closeStream() {
} }
} }
// The following synchronized methods are used whenever
// errorIndex or restartingNodeIndex is set. This is because
// check & set needs to be atomic. Simply reading variables
// does not require a synchronization. When responder is
// not running (e.g. during pipeline recovery), there is no
// need to use these methods.
/** Set the error node index. Called by responder */
synchronized void setErrorIndex(int idx) {
errorIndex = idx;
}
/** Set the restarting node index. Called by responder */
synchronized void setRestartingNodeIndex(int idx) {
restartingNodeIndex = idx;
// If the data streamer has already set the primary node
// bad, clear it. It is likely that the write failed due to
// the DN shutdown. Even if it was a real failure, the pipeline
// recovery will take care of it.
errorIndex = -1;
}
/**
* This method is used when no explicit error report was received,
* but something failed. When the primary node is a suspect or
* unsure about the cause, the primary node is marked as failed.
*/
synchronized void tryMarkPrimaryDatanodeFailed() {
// There should be no existing error and no ongoing restart.
if ((errorIndex == -1) && (restartingNodeIndex == -1)) {
errorIndex = 0;
}
}
/**
* Examine whether it is worth waiting for a node to restart.
* @param index the node index
*/
boolean shouldWaitForRestart(int index) {
// Only one node in the pipeline.
if (nodes.length == 1) {
return true;
}
// Is it a local node?
InetAddress addr = null;
try {
addr = InetAddress.getByName(nodes[index].getIpAddr());
} catch (java.net.UnknownHostException e) {
// we are passing an ip address. this should not happen.
assert false;
}
if (addr != null && NetUtils.isLocalAddress(addr)) {
return true;
}
return false;
}
// //
// Processes responses from the datanodes. A packet is removed // Processes responses from the datanodes. A packet is removed
// from the ackQueue when its response arrives. // from the ackQueue when its response arrives.
@ -732,8 +802,20 @@ public void run() {
// processes response status from datanodes. // processes response status from datanodes.
for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) { for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
final Status reply = ack.getReply(i); final Status reply = ack.getReply(i);
// Restart will not be treated differently unless it is
// the local node or the only one in the pipeline.
if (PipelineAck.isRestartOOBStatus(reply) &&
shouldWaitForRestart(i)) {
restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
Time.now();
setRestartingNodeIndex(i);
String message = "A datanode is restarting: " + targets[i];
DFSClient.LOG.info(message);
throw new IOException(message);
}
// node error
if (reply != SUCCESS) { if (reply != SUCCESS) {
errorIndex = i; // first bad datanode setErrorIndex(i); // first bad datanode
throw new IOException("Bad response " + reply + throw new IOException("Bad response " + reply +
" for block " + block + " for block " + block +
" from datanode " + " from datanode " +
@ -782,12 +864,16 @@ public void run() {
setLastException((IOException)e); setLastException((IOException)e);
} }
hasError = true; hasError = true;
errorIndex = errorIndex==-1 ? 0 : errorIndex; // If no explicit error report was received, mark the primary
// node as failed.
tryMarkPrimaryDatanodeFailed();
synchronized (dataQueue) { synchronized (dataQueue) {
dataQueue.notifyAll(); dataQueue.notifyAll();
} }
DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception " if (restartingNodeIndex == -1) {
+ " for block " + block, e); DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
+ " for block " + block, e);
}
responderClosed = true; responderClosed = true;
} }
} }
@ -1006,6 +1092,24 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
boolean success = false; boolean success = false;
long newGS = 0L; long newGS = 0L;
while (!success && !streamerClosed && dfsClient.clientRunning) { while (!success && !streamerClosed && dfsClient.clientRunning) {
// Sleep before reconnect if a dn is restarting.
// This process will be repeated until the deadline or the datanode
// starts back up.
if (restartingNodeIndex >= 0) {
// 4 seconds or the configured deadline period, whichever is shorter.
// This is the retry interval and recovery will be retried in this
// interval until timeout or success.
long delay = Math.min(dfsClient.getConf().datanodeRestartTimeout,
4000L);
try {
Thread.sleep(delay);
} catch (InterruptedException ie) {
lastException.set(new IOException("Interrupted while waiting for " +
"datanode to restart. " + nodes[restartingNodeIndex]));
streamerClosed = true;
return false;
}
}
boolean isRecovery = hasError; boolean isRecovery = hasError;
// remove bad datanode from list of datanodes. // remove bad datanode from list of datanodes.
// If errorIndex was not set (i.e. appends), then do not remove // If errorIndex was not set (i.e. appends), then do not remove
@ -1042,7 +1146,24 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
setPipeline(newnodes, newStorageIDs); setPipeline(newnodes, newStorageIDs);
hasError = false; // Just took care of a node error while waiting for a node restart
if (restartingNodeIndex >= 0) {
// If the error came from a node further away than the restarting
// node, the restart must have been complete.
if (errorIndex > restartingNodeIndex) {
restartingNodeIndex = -1;
} else if (errorIndex < restartingNodeIndex) {
// the node index has shifted.
restartingNodeIndex--;
} else {
// this shouldn't happen...
assert false;
}
}
if (restartingNodeIndex == -1) {
hasError = false;
}
lastException.set(null); lastException.set(null);
errorIndex = -1; errorIndex = -1;
} }
@ -1071,7 +1192,34 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
} else { } else {
success = createBlockOutputStream(nodes, newGS, isRecovery); success = createBlockOutputStream(nodes, newGS, isRecovery);
} }
}
if (restartingNodeIndex >= 0) {
assert hasError == true;
// check errorIndex set above
if (errorIndex == restartingNodeIndex) {
// ignore, if came from the restarting node
errorIndex = -1;
}
// still within the deadline
if (Time.now() < restartDeadline) {
continue; // with in the deadline
}
// expired. declare the restarting node dead
restartDeadline = 0;
int expiredNodeIndex = restartingNodeIndex;
restartingNodeIndex = -1;
DFSClient.LOG.warn("Datanode did not restart in time: " +
nodes[expiredNodeIndex]);
// Mark the restarting node as failed. If there is any other failed
// node during the last pipeline construction attempt, it will not be
// overwritten/dropped. In this case, the restarting node will get
// excluded in the following attempt, if it still does not come up.
if (errorIndex == -1) {
errorIndex = expiredNodeIndex;
}
// From this point on, normal pipeline recovery applies.
}
} // while
if (success) { if (success) {
// update pipeline at the namenode // update pipeline at the namenode
@ -1149,6 +1297,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS,
} }
Status pipelineStatus = SUCCESS; Status pipelineStatus = SUCCESS;
String firstBadLink = ""; String firstBadLink = "";
boolean checkRestart = false;
if (DFSClient.LOG.isDebugEnabled()) { if (DFSClient.LOG.isDebugEnabled()) {
for (int i = 0; i < nodes.length; i++) { for (int i = 0; i < nodes.length; i++) {
DFSClient.LOG.debug("pipeline = " + nodes[i]); DFSClient.LOG.debug("pipeline = " + nodes[i]);
@ -1197,6 +1346,16 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS,
pipelineStatus = resp.getStatus(); pipelineStatus = resp.getStatus();
firstBadLink = resp.getFirstBadLink(); firstBadLink = resp.getFirstBadLink();
// Got an restart OOB ack.
// If a node is already restarting, this status is not likely from
// the same node. If it is from a different node, it is not
// from the local datanode. Thus it is safe to treat this as a
// regular node error.
if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
restartingNodeIndex == -1) {
checkRestart = true;
throw new IOException("A datanode is restarting.");
}
if (pipelineStatus != SUCCESS) { if (pipelineStatus != SUCCESS) {
if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) { if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) {
throw new InvalidBlockTokenException( throw new InvalidBlockTokenException(
@ -1210,9 +1369,12 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS,
assert null == blockStream : "Previous blockStream unclosed"; assert null == blockStream : "Previous blockStream unclosed";
blockStream = out; blockStream = out;
result = true; // success result = true; // success
restartingNodeIndex = -1;
hasError = false;
} catch (IOException ie) { } catch (IOException ie) {
DFSClient.LOG.info("Exception in createBlockOutputStream", ie); if (restartingNodeIndex == -1) {
DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
}
if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, " DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+ "encryption key was invalid when connecting to " + "encryption key was invalid when connecting to "
@ -1235,8 +1397,18 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS,
} }
} }
} else { } else {
assert checkRestart == false;
errorIndex = 0; errorIndex = 0;
} }
// Check whether there is a restart worth waiting for.
if (checkRestart && shouldWaitForRestart(errorIndex)) {
restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
Time.now();
restartingNodeIndex = errorIndex;
errorIndex = -1;
DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
nodes[restartingNodeIndex]);
}
hasError = true; hasError = true;
setLastException(ie); setLastException(ie);
result = false; // error result = false; // error

View File

@ -61,6 +61,7 @@
import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -68,11 +69,12 @@
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@ -1037,6 +1039,14 @@ public void finalizeUpgrade() throws IOException {
dfs.finalizeUpgrade(); dfs.finalizeUpgrade();
} }
/**
* Rolling upgrade: start/finalize/query.
*/
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
throws IOException {
return dfs.rollingUpgrade(action);
}
/* /*
* Requests the namenode to dump data strcutures into specified * Requests the namenode to dump data strcutures into specified
* file. * file.

View File

@ -127,4 +127,22 @@ BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
*/ */
HdfsBlocksMetadata getHdfsBlocksMetadata(String blockPoolId, HdfsBlocksMetadata getHdfsBlocksMetadata(String blockPoolId,
long []blockIds, List<Token<BlockTokenIdentifier>> tokens) throws IOException; long []blockIds, List<Token<BlockTokenIdentifier>> tokens) throws IOException;
/**
* Shuts down a datanode.
*
* @param forUpgrade If true, data node does extra prep work before shutting
* down. The work includes advising clients to wait and saving
* certain states for quick restart. This should only be used when
* the stored data will remain the same during upgrade/restart.
* @throws IOException
*/
void shutdownDatanode(boolean forUpgrade) throws IOException;
/**
* Obtains datanode info
*
* @return software/config version and uptime of the datanode
*/
DatanodeLocalInfo getDatanodeInfo() throws IOException;
} }

View File

@ -39,6 +39,7 @@
import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@ -763,6 +764,15 @@ public boolean restoreFailedStorage(String arg)
@Idempotent @Idempotent
public void finalizeUpgrade() throws IOException; public void finalizeUpgrade() throws IOException;
/**
* Rolling upgrade operations.
* @param action either query, start or finailze.
* @return rolling upgrade information.
*/
@Idempotent
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
throws IOException;
/** /**
* @return CorruptFileBlocks, containing a list of corrupt files (with * @return CorruptFileBlocks, containing a list of corrupt files (with
* duplicates if there is more than one corrupt block in a file) * duplicates if there is more than one corrupt block in a file)

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Locally available datanode information
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DatanodeLocalInfo {
private String softwareVersion;
private String configVersion;
private long uptime; // datanode uptime in seconds.
public DatanodeLocalInfo(String softwareVersion,
String configVersion, long uptime) {
this.softwareVersion = softwareVersion;
this.configVersion = configVersion;
this.uptime = uptime;
}
/** get software version */
public String getSoftwareVersion() {
return this.softwareVersion;
}
/** get config version */
public String getConfigVersion() {
return this.configVersion;
}
/** get uptime */
public long getUptime() {
return this.uptime;
}
/** A formatted string for printing the status of the DataNode. */
public String getDatanodeLocalReport() {
StringBuilder buffer = new StringBuilder();
buffer.append("Uptime: " + getUptime());
buffer.append(", Software version: " + getSoftwareVersion());
buffer.append(", Config version: " + getConfigVersion());
return buffer.toString();
}
}

View File

@ -17,12 +17,19 @@
*/ */
package org.apache.hadoop.hdfs.protocol; package org.apache.hadoop.hdfs.protocol;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
/************************************ /************************************
* Some handy constants * Some handy constants
@ -79,6 +86,24 @@ public static enum SafeModeAction {
SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET;
} }
public static enum RollingUpgradeAction {
QUERY, PREPARE, FINALIZE;
private static final Map<String, RollingUpgradeAction> MAP
= new HashMap<String, RollingUpgradeAction>();
static {
MAP.put("", QUERY);
for(RollingUpgradeAction a : values()) {
MAP.put(a.name(), a);
}
}
/** Covert the given String to a RollingUpgradeAction. */
public static RollingUpgradeAction fromString(String s) {
return MAP.get(s.toUpperCase());
}
}
// type of the datanode report // type of the datanode report
public static enum DatanodeReportType { public static enum DatanodeReportType {
ALL, LIVE, DEAD ALL, LIVE, DEAD
@ -103,13 +128,6 @@ public static enum DatanodeReportType {
*/ */
public static final String HA_DT_SERVICE_PREFIX = "ha-hdfs:"; public static final String HA_DT_SERVICE_PREFIX = "ha-hdfs:";
/**
* Please see {@link LayoutVersion} on adding new layout version.
*/
public static final int LAYOUT_VERSION = LayoutVersion
.getCurrentLayoutVersion();
/** /**
* Path components that are reserved in HDFS. * Path components that are reserved in HDFS.
* <p> * <p>
@ -120,6 +138,20 @@ public static enum DatanodeReportType {
FSDirectory.DOT_RESERVED_STRING FSDirectory.DOT_RESERVED_STRING
}; };
/**
* Current layout version for NameNode.
* Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.
*/
public static final int NAMENODE_LAYOUT_VERSION
= NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
/**
* Current layout version for DataNode.
* Please see {@link DataNodeLayoutVersion.Feature} on adding new layout version.
*/
public static final int DATANODE_LAYOUT_VERSION
= DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
/** /**
* A special path component contained in the path for a snapshot file/dir * A special path component contained in the path for a snapshot file/dir
*/ */

View File

@ -17,9 +17,10 @@
*/ */
package org.apache.hadoop.hdfs.protocol; package org.apache.hadoop.hdfs.protocol;
import java.util.EnumSet; import java.util.Comparator;
import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -43,7 +44,6 @@
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class LayoutVersion { public class LayoutVersion {
/** /**
* Version in which HDFS-2991 was fixed. This bug caused OP_ADD to * Version in which HDFS-2991 was fixed. This bug caused OP_ADD to
* sometimes be skipped for append() calls. If we see such a case when * sometimes be skipped for append() calls. If we see such a case when
@ -54,19 +54,27 @@ public class LayoutVersion {
public static final int BUGFIX_HDFS_2991_VERSION = -40; public static final int BUGFIX_HDFS_2991_VERSION = -40;
/** /**
* Enums for features that change the layout version. * The interface to be implemented by NameNode and DataNode layout features
*/
public interface LayoutFeature {
public FeatureInfo getInfo();
}
/**
* Enums for features that change the layout version before rolling
* upgrade is supported.
* <br><br> * <br><br>
* To add a new layout version: * To add a new layout version:
* <ul> * <ul>
* <li>Define a new enum constant with a short enum name, the new layout version * <li>Define a new enum constant with a short enum name, the new layout version
* and description of the added feature.</li> * and description of the added feature.</li>
* <li>When adding a layout version with an ancestor that is not same as * <li>When adding a layout version with an ancestor that is not same as
* its immediate predecessor, use the constructor where a spacific ancestor * its immediate predecessor, use the constructor where a specific ancestor
* can be passed. * can be passed.
* </li> * </li>
* </ul> * </ul>
*/ */
public static enum Feature { public static enum Feature implements LayoutFeature {
NAMESPACE_QUOTA(-16, "Support for namespace quotas"), NAMESPACE_QUOTA(-16, "Support for namespace quotas"),
FILE_ACCESS_TIME(-17, "Support for access time on files"), FILE_ACCESS_TIME(-17, "Support for access time on files"),
DISKSPACE_QUOTA(-18, "Support for disk space quotas"), DISKSPACE_QUOTA(-18, "Support for disk space quotas"),
@ -119,12 +127,8 @@ public static enum Feature {
RESERVED_REL2_4_0(-54, -51, "Reserved for release 2.4.0", true, RESERVED_REL2_4_0(-54, -51, "Reserved for release 2.4.0", true,
PROTOBUF_FORMAT, EXTENDED_ACL); PROTOBUF_FORMAT, EXTENDED_ACL);
final int lv; private final FeatureInfo info;
final int ancestorLV;
final String description;
final boolean reserved;
final Feature[] specialFeatures;
/** /**
* Feature that is added at layout version {@code lv} - 1. * Feature that is added at layout version {@code lv} - 1.
* @param lv new layout version with the addition of this feature * @param lv new layout version with the addition of this feature
@ -140,16 +144,35 @@ public static enum Feature {
* @param ancestorLV layout version from which the new lv is derived from. * @param ancestorLV layout version from which the new lv is derived from.
* @param description description of the feature * @param description description of the feature
* @param reserved true when this is a layout version reserved for previous * @param reserved true when this is a layout version reserved for previous
* verions * version
* @param features set of features that are to be enabled for this version * @param features set of features that are to be enabled for this version
*/ */
Feature(final int lv, final int ancestorLV, final String description, Feature(final int lv, final int ancestorLV, final String description,
boolean reserved, Feature... features) { boolean reserved, Feature... features) {
info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
}
@Override
public FeatureInfo getInfo() {
return info;
}
}
/** Feature information. */
public static class FeatureInfo {
private final int lv;
private final int ancestorLV;
private final String description;
private final boolean reserved;
private final LayoutFeature[] specialFeatures;
public FeatureInfo(final int lv, final int ancestorLV, final String description,
boolean reserved, LayoutFeature... specialFeatures) {
this.lv = lv; this.lv = lv;
this.ancestorLV = ancestorLV; this.ancestorLV = ancestorLV;
this.description = description; this.description = description;
this.reserved = reserved; this.reserved = reserved;
specialFeatures = features; this.specialFeatures = specialFeatures;
} }
/** /**
@ -179,80 +202,91 @@ public String getDescription() {
public boolean isReservedForOldRelease() { public boolean isReservedForOldRelease() {
return reserved; return reserved;
} }
public LayoutFeature[] getSpecialFeatures() {
return specialFeatures;
}
} }
// Build layout version and corresponding feature matrix static class LayoutFeatureComparator implements Comparator<LayoutFeature> {
static final Map<Integer, EnumSet<Feature>>map = @Override
new HashMap<Integer, EnumSet<Feature>>(); public int compare(LayoutFeature arg0, LayoutFeature arg1) {
return arg0.getInfo().getLayoutVersion()
// Static initialization - arg1.getInfo().getLayoutVersion();
static { }
initMap();
} }
/** public static void updateMap(Map<Integer, SortedSet<LayoutFeature>> map,
* Initialize the map of a layout version and EnumSet of {@link Feature}s LayoutFeature[] features) {
* supported.
*/
private static void initMap() {
// Go through all the enum constants and build a map of // Go through all the enum constants and build a map of
// LayoutVersion <-> EnumSet of all supported features in that LayoutVersion // LayoutVersion <-> Set of all supported features in that LayoutVersion
for (Feature f : Feature.values()) { for (LayoutFeature f : features) {
EnumSet<Feature> ancestorSet = map.get(f.ancestorLV); final FeatureInfo info = f.getInfo();
SortedSet<LayoutFeature> ancestorSet = map.get(info.getAncestorLayoutVersion());
if (ancestorSet == null) { if (ancestorSet == null) {
ancestorSet = EnumSet.noneOf(Feature.class); // Empty enum set // Empty set
map.put(f.ancestorLV, ancestorSet); ancestorSet = new TreeSet<LayoutFeature>(new LayoutFeatureComparator());
map.put(info.getAncestorLayoutVersion(), ancestorSet);
} }
EnumSet<Feature> featureSet = EnumSet.copyOf(ancestorSet); SortedSet<LayoutFeature> featureSet = new TreeSet<LayoutFeature>(ancestorSet);
if (f.specialFeatures != null) { if (info.getSpecialFeatures() != null) {
for (Feature specialFeature : f.specialFeatures) { for (LayoutFeature specialFeature : info.getSpecialFeatures()) {
featureSet.add(specialFeature); featureSet.add(specialFeature);
} }
} }
featureSet.add(f); featureSet.add(f);
map.put(f.lv, featureSet); map.put(info.getLayoutVersion(), featureSet);
} }
} }
/** /**
* Gets formatted string that describes {@link LayoutVersion} information. * Gets formatted string that describes {@link LayoutVersion} information.
*/ */
public static String getString() { public String getString(Map<Integer, SortedSet<LayoutFeature>> map,
LayoutFeature[] values) {
final StringBuilder buf = new StringBuilder(); final StringBuilder buf = new StringBuilder();
buf.append("Feature List:\n"); buf.append("Feature List:\n");
for (Feature f : Feature.values()) { for (LayoutFeature f : values) {
final FeatureInfo info = f.getInfo();
buf.append(f).append(" introduced in layout version ") buf.append(f).append(" introduced in layout version ")
.append(f.lv).append(" ("). .append(info.getLayoutVersion()).append(" (")
append(f.description).append(")\n"); .append(info.getDescription()).append(")\n");
} }
buf.append("\n\nLayoutVersion and supported features:\n"); buf.append("\n\nLayoutVersion and supported features:\n");
for (Feature f : Feature.values()) { for (LayoutFeature f : values) {
buf.append(f.lv).append(": ").append(map.get(f.lv)) final FeatureInfo info = f.getInfo();
.append("\n"); buf.append(info.getLayoutVersion()).append(": ")
.append(map.get(info.getLayoutVersion())).append("\n");
} }
return buf.toString(); return buf.toString();
} }
/** /**
* Returns true if a given feature is supported in the given layout version * Returns true if a given feature is supported in the given layout version
* @param map layout feature map
* @param f Feature * @param f Feature
* @param lv LayoutVersion * @param lv LayoutVersion
* @return true if {@code f} is supported in layout version {@code lv} * @return true if {@code f} is supported in layout version {@code lv}
*/ */
public static boolean supports(final Feature f, final int lv) { public static boolean supports(Map<Integer, SortedSet<LayoutFeature>> map,
final EnumSet<Feature> set = map.get(lv); final LayoutFeature f, final int lv) {
final SortedSet<LayoutFeature> set = map.get(lv);
return set != null && set.contains(f); return set != null && set.contains(f);
} }
/** /**
* Get the current layout version * Get the current layout version
*/ */
public static int getCurrentLayoutVersion() { public static int getCurrentLayoutVersion(LayoutFeature[] features) {
Feature[] values = Feature.values(); return getLastNonReservedFeature(features).getInfo().getLayoutVersion();
for (int i = values.length -1; i >= 0; i--) { }
if (!values[i].isReservedForOldRelease()) {
return values[i].lv; static LayoutFeature getLastNonReservedFeature(LayoutFeature[] features) {
for (int i = features.length -1; i >= 0; i--) {
final FeatureInfo info = features[i].getInfo();
if (!info.isReservedForOldRelease()) {
return features[i];
} }
} }
throw new AssertionError("All layout versions are reserved."); throw new AssertionError("All layout versions are reserved.");

View File

@ -0,0 +1,36 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception related to rolling upgrade.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RollingUpgradeException extends IOException {
private static final long serialVersionUID = 1L;
public RollingUpgradeException(String msg) {
super(msg);
}
}

View File

@ -0,0 +1,127 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol;
import java.util.Date;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Rolling upgrade information
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RollingUpgradeInfo extends RollingUpgradeStatus {
private final long startTime;
private final long finalizeTime;
private boolean createdRollbackImages;
public RollingUpgradeInfo(String blockPoolId, boolean createdRollbackImages,
long startTime, long finalizeTime) {
super(blockPoolId);
this.createdRollbackImages = createdRollbackImages;
this.startTime = startTime;
this.finalizeTime = finalizeTime;
}
public boolean createdRollbackImages() {
return createdRollbackImages;
}
public void setCreatedRollbackImages(boolean created) {
this.createdRollbackImages = created;
}
public boolean isStarted() {
return startTime != 0;
}
/** @return The rolling upgrade starting time. */
public long getStartTime() {
return startTime;
}
public boolean isFinalized() {
return finalizeTime != 0;
}
public long getFinalizeTime() {
return finalizeTime;
}
@Override
public int hashCode() {
//only use lower 32 bits
return super.hashCode() ^ (int)startTime ^ (int)finalizeTime;
}
@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
} else if (obj == null || !(obj instanceof RollingUpgradeInfo)) {
return false;
}
final RollingUpgradeInfo that = (RollingUpgradeInfo)obj;
return super.equals(that)
&& this.startTime == that.startTime
&& this.finalizeTime == that.finalizeTime;
}
@Override
public String toString() {
return super.toString()
+ "\n Start Time: " + (startTime == 0? "<NOT STARTED>": timestamp2String(startTime))
+ "\n Finalize Time: " + (finalizeTime == 0? "<NOT FINALIZED>": timestamp2String(finalizeTime));
}
private static String timestamp2String(long timestamp) {
return new Date(timestamp) + " (=" + timestamp + ")";
}
public static class Bean {
private final String blockPoolId;
private final long startTime;
private final long finalizeTime;
private final boolean createdRollbackImages;
public Bean(RollingUpgradeInfo f) {
this.blockPoolId = f.getBlockPoolId();
this.startTime = f.startTime;
this.finalizeTime = f.finalizeTime;
this.createdRollbackImages = f.createdRollbackImages();
}
public String getBlockPoolId() {
return blockPoolId;
}
public long getStartTime() {
return startTime;
}
public long getFinalizeTime() {
return finalizeTime;
}
public boolean isCreatedRollbackImages() {
return createdRollbackImages;
}
}
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Rolling upgrade status
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RollingUpgradeStatus {
private String blockPoolId;
public RollingUpgradeStatus(String blockPoolId) {
this.blockPoolId = blockPoolId;
}
public String getBlockPoolId() {
return blockPoolId;
}
@Override
public int hashCode() {
return blockPoolId.hashCode();
}
@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
} else if (obj == null || !(obj instanceof RollingUpgradeStatus)) {
return false;
}
final RollingUpgradeStatus that = (RollingUpgradeStatus)obj;
return this.blockPoolId.equals(that.blockPoolId);
}
@Override
public String toString() {
return " Block Pool ID: " + blockPoolId;
}
}

View File

@ -26,10 +26,12 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_DEFAULT;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import com.google.protobuf.TextFormat; import com.google.protobuf.TextFormat;
/** Pipeline Acknowledgment **/ /** Pipeline Acknowledgment **/
@ -38,6 +40,21 @@
public class PipelineAck { public class PipelineAck {
PipelineAckProto proto; PipelineAckProto proto;
public final static long UNKOWN_SEQNO = -2; public final static long UNKOWN_SEQNO = -2;
final static int OOB_START = Status.OOB_RESTART_VALUE; // the first OOB type
final static int OOB_END = Status.OOB_RESERVED3_VALUE; // the last OOB type
final static int NUM_OOB_TYPES = OOB_END - OOB_START + 1;
// place holder for timeout value of each OOB type
final static long[] OOB_TIMEOUT;
static {
OOB_TIMEOUT = new long[NUM_OOB_TYPES];
HdfsConfiguration conf = new HdfsConfiguration();
String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
for (int i = 0; i < NUM_OOB_TYPES; i++) {
OOB_TIMEOUT[i] = (i < ele.length) ? Long.valueOf(ele[i]) : 0;
}
}
/** default constructor **/ /** default constructor **/
public PipelineAck() { public PipelineAck() {
@ -103,14 +120,57 @@ public long getDownstreamAckTimeNanos() {
* @return true if all statuses are SUCCESS * @return true if all statuses are SUCCESS
*/ */
public boolean isSuccess() { public boolean isSuccess() {
for (DataTransferProtos.Status reply : proto.getStatusList()) { for (Status reply : proto.getStatusList()) {
if (reply != DataTransferProtos.Status.SUCCESS) { if (reply != Status.SUCCESS) {
return false; return false;
} }
} }
return true; return true;
} }
/**
* Returns the OOB status if this ack contains one.
* @return null if it is not an OOB ack.
*/
public Status getOOBStatus() {
// Normal data transfer acks will have a valid sequence number, so
// this will return right away in most cases.
if (getSeqno() != UNKOWN_SEQNO) {
return null;
}
for (Status reply : proto.getStatusList()) {
// The following check is valid because protobuf guarantees to
// preserve the ordering of enum elements.
if (reply.getNumber() >= OOB_START && reply.getNumber() <= OOB_END) {
return reply;
}
}
return null;
}
/**
* Get the timeout to be used for transmitting the OOB type
* @return the timeout in milliseconds
*/
public static long getOOBTimeout(Status status) throws IOException {
int index = status.getNumber() - OOB_START;
if (index >= 0 && index < NUM_OOB_TYPES) {
return OOB_TIMEOUT[index];
}
// Not an OOB.
throw new IOException("Not an OOB status: " + status);
}
/** Get the Restart OOB ack status */
public static Status getRestartOOBStatus() {
return Status.OOB_RESTART;
}
/** return true if it is the restart OOB status code */
public static boolean isRestartOOBStatus(Status st) {
return st.equals(Status.OOB_RESTART);
}
/**** Writable interface ****/ /**** Writable interface ****/
public void readFields(InputStream in) throws IOException { public void readFields(InputStream in) throws IOException {
proto = PipelineAckProto.parseFrom(vintPrefixed(in)); proto = PipelineAckProto.parseFrom(vintPrefixed(in));

View File

@ -30,6 +30,8 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetDatanodeInfoRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetDatanodeInfoResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto.Builder; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto.Builder;
@ -37,10 +39,13 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.VersionInfo;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
@ -59,6 +64,8 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
RefreshNamenodesResponseProto.newBuilder().build(); RefreshNamenodesResponseProto.newBuilder().build();
private final static DeleteBlockPoolResponseProto DELETE_BLOCKPOOL_RESP = private final static DeleteBlockPoolResponseProto DELETE_BLOCKPOOL_RESP =
DeleteBlockPoolResponseProto.newBuilder().build(); DeleteBlockPoolResponseProto.newBuilder().build();
private final static ShutdownDatanodeResponseProto SHUTDOWN_DATANODE_RESP =
ShutdownDatanodeResponseProto.newBuilder().build();
private final ClientDatanodeProtocol impl; private final ClientDatanodeProtocol impl;
@ -151,4 +158,28 @@ public GetHdfsBlockLocationsResponseProto getHdfsBlockLocations(
builder.addAllVolumeIndexes(resp.getVolumeIndexes()); builder.addAllVolumeIndexes(resp.getVolumeIndexes());
return builder.build(); return builder.build();
} }
@Override
public ShutdownDatanodeResponseProto shutdownDatanode(
RpcController unused, ShutdownDatanodeRequestProto request)
throws ServiceException {
try {
impl.shutdownDatanode(request.getForUpgrade());
} catch (IOException e) {
throw new ServiceException(e);
}
return SHUTDOWN_DATANODE_RESP;
}
public GetDatanodeInfoResponseProto getDatanodeInfo(RpcController unused,
GetDatanodeInfoRequestProto request) throws ServiceException {
GetDatanodeInfoResponseProto res;
try {
res = GetDatanodeInfoResponseProto.newBuilder()
.setLocalInfo(PBHelper.convert(impl.getDatanodeInfo())).build();
} catch (IOException e) {
throw new ServiceException(e);
}
return res;
}
} }

View File

@ -35,16 +35,20 @@
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo; import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol; import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetDatanodeInfoRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetDatanodeInfoResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtobufHelper;
@ -81,6 +85,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
private final ClientDatanodeProtocolPB rpcProxy; private final ClientDatanodeProtocolPB rpcProxy;
private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES = private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES =
RefreshNamenodesRequestProto.newBuilder().build(); RefreshNamenodesRequestProto.newBuilder().build();
private final static GetDatanodeInfoRequestProto VOID_GET_DATANODE_INFO =
GetDatanodeInfoRequestProto.newBuilder().build();
public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid, public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
Configuration conf, int socketTimeout, boolean connectToDnViaHostname, Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
@ -253,4 +259,27 @@ public HdfsBlocksMetadata getHdfsBlocksMetadata(String blockPoolId,
return new HdfsBlocksMetadata(blockPoolId, blockIds, return new HdfsBlocksMetadata(blockPoolId, blockIds,
volumeIds, volumeIndexes); volumeIds, volumeIndexes);
} }
@Override
public void shutdownDatanode(boolean forUpgrade) throws IOException {
ShutdownDatanodeRequestProto request = ShutdownDatanodeRequestProto
.newBuilder().setForUpgrade(forUpgrade).build();
try {
rpcProxy.shutdownDatanode(NULL_CONTROLLER, request);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public DatanodeLocalInfo getDatanodeInfo() throws IOException {
GetDatanodeInfoResponseProto response;
try {
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO);
return PBHelper.convert(response.getLocalInfo());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
} }

View File

@ -22,11 +22,12 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.Options.Rename; import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@ -34,7 +35,7 @@
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@ -53,10 +54,10 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
@ -112,28 +113,28 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
@ -148,6 +149,8 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
@ -711,6 +714,22 @@ public FinalizeUpgradeResponseProto finalizeUpgrade(RpcController controller,
} }
} }
@Override
public RollingUpgradeResponseProto rollingUpgrade(RpcController controller,
RollingUpgradeRequestProto req) throws ServiceException {
try {
final RollingUpgradeInfo info = server.rollingUpgrade(
PBHelper.convert(req.getAction()));
final RollingUpgradeResponseProto.Builder b = RollingUpgradeResponseProto.newBuilder();
if (info != null) {
b.setRollingUpgradeInfo(PBHelper.convert(info));
}
return b.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override @Override
public ListCorruptFileBlocksResponseProto listCorruptFileBlocks( public ListCorruptFileBlocksResponseProto listCorruptFileBlocks(
RpcController controller, ListCorruptFileBlocksRequestProto req) RpcController controller, ListCorruptFileBlocksRequestProto req)

View File

@ -40,8 +40,9 @@
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@ -50,12 +51,13 @@
import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@ -66,8 +68,8 @@
import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
@ -106,19 +108,19 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
@ -127,6 +129,8 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
@ -644,6 +648,21 @@ public void finalizeUpgrade() throws IOException {
} }
} }
@Override
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
.setAction(PBHelper.convert(action)).build();
try {
final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
if (proto.hasRollingUpgradeInfo()) {
return PBHelper.convert(proto.getRollingUpgradeInfo());
}
return null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override @Override
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie) public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException { throws IOException {

View File

@ -34,6 +34,7 @@
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
@ -184,7 +185,12 @@ public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
cmds[index] = PBHelper.convert(p); cmds[index] = PBHelper.convert(p);
index++; index++;
} }
return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus())); RollingUpgradeStatus rollingUpdateStatus = null;
if (resp.hasRollingUpgradeStatus()) {
rollingUpdateStatus = PBHelper.convert(resp.getRollingUpgradeStatus());
}
return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus()),
rollingUpdateStatus);
} }
@Override @Override

View File

@ -23,6 +23,7 @@
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
@ -121,6 +122,12 @@ public HeartbeatResponseProto sendHeartbeat(RpcController controller,
} }
} }
builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState())); builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState()));
RollingUpgradeStatus rollingUpdateStatus = response
.getRollingUpdateStatus();
if (rollingUpdateStatus != null) {
builder.setRollingUpgradeStatus(PBHelper
.convertRollingUpgradeStatus(rollingUpdateStatus));
}
return builder.build(); return builder.build();
} }

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats; import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@ -54,17 +55,20 @@
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -77,6 +81,7 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
@ -85,6 +90,8 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
@ -117,6 +124,7 @@
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
@ -135,6 +143,7 @@
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
@ -149,6 +158,7 @@
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
@ -254,9 +264,9 @@ public static StorageInfoProto convert(StorageInfo info) {
.setNamespceID(info.getNamespaceID()).build(); .setNamespceID(info.getNamespaceID()).build();
} }
public static StorageInfo convert(StorageInfoProto info) { public static StorageInfo convert(StorageInfoProto info, NodeType type) {
return new StorageInfo(info.getLayoutVersion(), info.getNamespceID(), return new StorageInfo(info.getLayoutVersion(), info.getNamespceID(),
info.getClusterID(), info.getCTime()); info.getClusterID(), info.getCTime(), type);
} }
public static NamenodeRegistrationProto convert(NamenodeRegistration reg) { public static NamenodeRegistrationProto convert(NamenodeRegistration reg) {
@ -267,8 +277,9 @@ public static NamenodeRegistrationProto convert(NamenodeRegistration reg) {
} }
public static NamenodeRegistration convert(NamenodeRegistrationProto reg) { public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
StorageInfo si = convert(reg.getStorageInfo(), NodeType.NAME_NODE);
return new NamenodeRegistration(reg.getRpcAddress(), reg.getHttpAddress(), return new NamenodeRegistration(reg.getRpcAddress(), reg.getHttpAddress(),
convert(reg.getStorageInfo()), convert(reg.getRole())); si, convert(reg.getRole()));
} }
// DatanodeId // DatanodeId
@ -400,9 +411,9 @@ public static CheckpointSignatureProto convert(CheckpointSignature s) {
} }
public static CheckpointSignature convert(CheckpointSignatureProto s) { public static CheckpointSignature convert(CheckpointSignatureProto s) {
return new CheckpointSignature(PBHelper.convert(s.getStorageInfo()), StorageInfo si = PBHelper.convert(s.getStorageInfo(), NodeType.NAME_NODE);
s.getBlockPoolId(), s.getMostRecentCheckpointTxId(), return new CheckpointSignature(si, s.getBlockPoolId(),
s.getCurSegmentTxId()); s.getMostRecentCheckpointTxId(), s.getCurSegmentTxId());
} }
public static RemoteEditLogProto convert(RemoteEditLog log) { public static RemoteEditLogProto convert(RemoteEditLog log) {
@ -755,9 +766,9 @@ public static DatanodeRegistrationProto convert(
} }
public static DatanodeRegistration convert(DatanodeRegistrationProto proto) { public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
StorageInfo si = convert(proto.getStorageInfo(), NodeType.DATA_NODE);
return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()), return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto si, PBHelper.convert(proto.getKeys()), proto.getSoftwareVersion());
.getKeys()), proto.getSoftwareVersion());
} }
public static DatanodeCommand convert(DatanodeCommandProto proto) { public static DatanodeCommand convert(DatanodeCommandProto proto) {
@ -1448,6 +1459,59 @@ public static SafeModeAction convert(
} }
} }
public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
switch (a) {
case QUERY:
return RollingUpgradeActionProto.QUERY;
case PREPARE:
return RollingUpgradeActionProto.START;
case FINALIZE:
return RollingUpgradeActionProto.FINALIZE;
default:
throw new IllegalArgumentException("Unexpected value: " + a);
}
}
public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
switch (a) {
case QUERY:
return RollingUpgradeAction.QUERY;
case START:
return RollingUpgradeAction.PREPARE;
case FINALIZE:
return RollingUpgradeAction.FINALIZE;
default:
throw new IllegalArgumentException("Unexpected value: " + a);
}
}
public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
RollingUpgradeStatus status) {
return RollingUpgradeStatusProto.newBuilder()
.setBlockPoolId(status.getBlockPoolId())
.build();
}
public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
return new RollingUpgradeStatus(proto.getBlockPoolId());
}
public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
return RollingUpgradeInfoProto.newBuilder()
.setStatus(convertRollingUpgradeStatus(info))
.setCreatedRollbackImages(info.createdRollbackImages())
.setStartTime(info.getStartTime())
.setFinalizeTime(info.getFinalizeTime())
.build();
}
public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
RollingUpgradeStatusProto status = proto.getStatus();
return new RollingUpgradeInfo(status.getBlockPoolId(),
proto.getCreatedRollbackImages(),
proto.getStartTime(), proto.getFinalizeTime());
}
public static CorruptFileBlocks convert(CorruptFileBlocksProto c) { public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
if (c == null) if (c == null)
return null; return null;
@ -1903,6 +1967,19 @@ public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
return HdfsProtos.ChecksumTypeProto.valueOf(type.id); return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
} }
public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
builder.setSoftwareVersion(info.getSoftwareVersion());
builder.setConfigVersion(info.getConfigVersion());
builder.setUptime(info.getUptime());
return builder.build();
}
public static DatanodeLocalInfo convert(DatanodeLocalInfoProto proto) {
return new DatanodeLocalInfo(proto.getSoftwareVersion(),
proto.getConfigVersion(), proto.getUptime());
}
public static InputStream vintPrefixed(final InputStream input) public static InputStream vintPrefixed(final InputStream input)
throws IOException { throws IOException {
final int firstByte = input.read(); final int firstByte = input.read();

View File

@ -151,4 +151,6 @@ public ListenableFuture<Void> acceptRecovery(SegmentStateProto log,
* StringBuilder. This is displayed on the NN web UI. * StringBuilder. This is displayed on the NN web UI.
*/ */
public void appendReport(StringBuilder sb); public void appendReport(StringBuilder sb);
public ListenableFuture<Void> discardSegments(long startTxId);
} }

View File

@ -308,4 +308,13 @@ QuorumCall<AsyncLogger,Void> format(NamespaceInfo nsInfo) {
} }
return QuorumCall.create(calls); return QuorumCall.create(calls);
} }
public QuorumCall<AsyncLogger, Void> discardSegments(long startTxId) {
Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<Void> future = logger.discardSegments(startTxId);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
} }

View File

@ -564,6 +564,17 @@ public Void call() throws IOException {
}); });
} }
@Override
public ListenableFuture<Void> discardSegments(final long startTxId) {
return executor.submit(new Callable<Void>() {
@Override
public Void call() throws IOException {
getProxy().discardSegments(journalId, startTxId);
return null;
}
});
}
@Override @Override
public String toString() { public String toString() {
return InetAddresses.toAddrString(addr.getAddress()) + ':' + return InetAddresses.toAddrString(addr.getAddress()) + ':' +

View File

@ -79,6 +79,7 @@ public class QuorumJournalManager implements JournalManager {
// configurable. // configurable.
private static final int FORMAT_TIMEOUT_MS = 60000; private static final int FORMAT_TIMEOUT_MS = 60000;
private static final int HASDATA_TIMEOUT_MS = 60000; private static final int HASDATA_TIMEOUT_MS = 60000;
private static final int DISCARD_SEGMENTS_TIMEOUT_MS = 60000;
private final Configuration conf; private final Configuration conf;
private final URI uri; private final URI uri;
@ -492,4 +493,22 @@ AsyncLoggerSet getLoggerSetForTests() {
return loggers; return loggers;
} }
@Override
public void discardSegments(long startTxId) throws IOException {
QuorumCall<AsyncLogger, Void> call = loggers.discardSegments(startTxId);
try {
call.waitFor(loggers.size(), loggers.size(), 0,
DISCARD_SEGMENTS_TIMEOUT_MS, "discardSegments");
if (call.countExceptions() > 0) {
call.rethrowException(
"Could not perform discardSegments of one or more JournalNodes");
}
} catch (InterruptedException e) {
throw new IOException(
"Interrupted waiting for discardSegments() response");
} catch (TimeoutException e) {
throw new IOException(
"Timed out waiting for discardSegments() response");
}
}
} }

View File

@ -31,6 +31,7 @@
import org.apache.hadoop.hdfs.qjournal.server.JournalNode; import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
import org.apache.hadoop.hdfs.server.namenode.JournalManager; import org.apache.hadoop.hdfs.server.namenode.JournalManager;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.security.KerberosInfo; import org.apache.hadoop.security.KerberosInfo;
/** /**
@ -143,4 +144,12 @@ public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo,
*/ */
public void acceptRecovery(RequestInfo reqInfo, public void acceptRecovery(RequestInfo reqInfo,
SegmentStateProto stateToAccept, URL fromUrl) throws IOException; SegmentStateProto stateToAccept, URL fromUrl) throws IOException;
/**
* Discard journal segments whose first TxId is greater than or equal to the
* given txid.
*/
@Idempotent
public void discardSegments(String journalId, long startTxId)
throws IOException;
} }

View File

@ -17,11 +17,19 @@
*/ */
package org.apache.hadoop.hdfs.qjournal.protocolPB; package org.apache.hadoop.hdfs.qjournal.protocolPB;
import java.io.IOException;
import java.net.URL;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DiscardSegmentsRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DiscardSegmentsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
@ -39,8 +47,6 @@
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
@ -48,13 +54,11 @@
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo; import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.net.URL;
/** /**
* Implementation for protobuf service that forwards requests * Implementation for protobuf service that forwards requests
* received on {@link JournalProtocolPB} to the * received on {@link JournalProtocolPB} to the
@ -244,4 +248,16 @@ private RequestInfo convert(
reqInfo.hasCommittedTxId() ? reqInfo.hasCommittedTxId() ?
reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID); reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID);
} }
@Override
public DiscardSegmentsResponseProto discardSegments(
RpcController controller, DiscardSegmentsRequestProto request)
throws ServiceException {
try {
impl.discardSegments(convert(request.getJid()), request.getStartTxId());
return DiscardSegmentsResponseProto.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);
}
}
} }

View File

@ -23,9 +23,12 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DiscardSegmentsRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
@ -39,7 +42,6 @@
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
@ -277,4 +279,16 @@ public boolean isMethodSupported(String methodName) throws IOException {
RPC.getProtocolVersion(QJournalProtocolPB.class), methodName); RPC.getProtocolVersion(QJournalProtocolPB.class), methodName);
} }
@Override
public void discardSegments(String journalId, long startTxId)
throws IOException {
try {
rpcProxy.discardSegments(NULL_CONTROLLER,
DiscardSegmentsRequestProto.newBuilder()
.setJid(convertJournalId(journalId)).setStartTxId(startTxId)
.build());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
} }

View File

@ -980,4 +980,10 @@ private void persistPaxosData(long segmentTxId,
} }
} }
} }
synchronized void discardSegments(long startTxId) throws IOException {
storage.getJournalManager().discardSegments(startTxId);
// we delete all the segments after the startTxId. let's reset committedTxnId
committedTxnId.set(startTxId - 1);
}
} }

View File

@ -285,4 +285,9 @@ public static void main(String[] args) throws Exception {
StringUtils.startupShutdownMessage(JournalNode.class, args, LOG); StringUtils.startupShutdownMessage(JournalNode.class, args, LOG);
System.exit(ToolRunner.run(new JournalNode(), args)); System.exit(ToolRunner.run(new JournalNode(), args));
} }
public void discardSegments(String journalId, long startTxId)
throws IOException {
getOrCreateJournal(journalId).discardSegments(startTxId);
}
} }

View File

@ -205,4 +205,9 @@ public void acceptRecovery(RequestInfo reqInfo, SegmentStateProto log,
.acceptRecovery(reqInfo, log, fromUrl); .acceptRecovery(reqInfo, log, fromUrl);
} }
@Override
public void discardSegments(String journalId, long startTxId)
throws IOException {
jn.discardSegments(journalId, startTxId);
}
} }

View File

@ -234,6 +234,22 @@ public int getPendingDataNodeMessageCount() {
*/ */
private boolean shouldPostponeBlocksFromFuture = false; private boolean shouldPostponeBlocksFromFuture = false;
/**
* Process replication queues asynchronously to allow namenode safemode exit
* and failover to be faster. HDFS-5496
*/
private Daemon replicationQueuesInitializer = null;
/**
* Number of blocks to process asychronously for replication queues
* initialization once aquired the namesystem lock. Remaining blocks will be
* processed again after aquiring lock again.
*/
private int numBlocksPerIteration;
/**
* Progress of the Replication queues initialisation.
*/
private double replicationQueuesInitProgress = 0.0;
/** for block replicas placement */ /** for block replicas placement */
private BlockPlacementPolicy blockplacement; private BlockPlacementPolicy blockplacement;
@ -310,6 +326,9 @@ public BlockManager(final Namesystem namesystem, final FSClusterStats stats,
this.maxNumBlocksToLog = this.maxNumBlocksToLog =
conf.getLong(DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY, conf.getLong(DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT); DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
this.numBlocksPerIteration = conf.getInt(
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT,
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT);
LOG.info("defaultReplication = " + defaultReplication); LOG.info("defaultReplication = " + defaultReplication);
LOG.info("maxReplication = " + maxReplication); LOG.info("maxReplication = " + maxReplication);
@ -2364,45 +2383,127 @@ private void invalidateCorruptReplicas(BlockInfo blk) {
*/ */
public void processMisReplicatedBlocks() { public void processMisReplicatedBlocks() {
assert namesystem.hasWriteLock(); assert namesystem.hasWriteLock();
stopReplicationInitializer();
long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0, nrPostponed = 0,
nrUnderConstruction = 0;
neededReplications.clear(); neededReplications.clear();
for (BlockInfo block : blocksMap.getBlocks()) { replicationQueuesInitializer = new Daemon() {
MisReplicationResult res = processMisReplicatedBlock(block);
if (LOG.isTraceEnabled()) { @Override
LOG.trace("block " + block + ": " + res); public void run() {
try {
processMisReplicatesAsync();
} catch (InterruptedException ie) {
LOG.info("Interrupted while processing replication queues.");
} catch (Exception e) {
LOG.error("Error while processing replication queues async", e);
}
} }
switch (res) { };
case UNDER_REPLICATED: replicationQueuesInitializer.setName("Replication Queue Initializer");
nrUnderReplicated++; replicationQueuesInitializer.start();
break; }
case OVER_REPLICATED:
nrOverReplicated++; /*
break; * Stop the ongoing initialisation of replication queues
case INVALID: */
nrInvalid++; private void stopReplicationInitializer() {
break; if (replicationQueuesInitializer != null) {
case POSTPONE: replicationQueuesInitializer.interrupt();
nrPostponed++; try {
postponeBlock(block); replicationQueuesInitializer.join();
break; } catch (final InterruptedException e) {
case UNDER_CONSTRUCTION: LOG.warn("Interrupted while waiting for replicationQueueInitializer. Returning..");
nrUnderConstruction++; return;
break; } finally {
case OK: replicationQueuesInitializer = null;
break;
default:
throw new AssertionError("Invalid enum value: " + res);
} }
} }
}
LOG.info("Total number of blocks = " + blocksMap.size());
LOG.info("Number of invalid blocks = " + nrInvalid); /*
LOG.info("Number of under-replicated blocks = " + nrUnderReplicated); * Since the BlocksMapGset does not throw the ConcurrentModificationException
LOG.info("Number of over-replicated blocks = " + nrOverReplicated + * and supports further iteration after modification to list, there is a
((nrPostponed > 0) ? ( " (" + nrPostponed + " postponed)") : "")); * chance of missing the newly added block while iterating. Since every
LOG.info("Number of blocks being written = " + nrUnderConstruction); * addition to blocksMap will check for mis-replication, missing mis-replication
* check for new blocks will not be a problem.
*/
private void processMisReplicatesAsync() throws InterruptedException {
long nrInvalid = 0, nrOverReplicated = 0;
long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
long startTimeMisReplicatedScan = Time.now();
Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
long totalBlocks = blocksMap.size();
replicationQueuesInitProgress = 0;
long totalProcessed = 0;
while (namesystem.isRunning() && !Thread.currentThread().isInterrupted()) {
int processed = 0;
namesystem.writeLockInterruptibly();
try {
while (processed < numBlocksPerIteration && blocksItr.hasNext()) {
BlockInfo block = blocksItr.next();
MisReplicationResult res = processMisReplicatedBlock(block);
if (LOG.isTraceEnabled()) {
LOG.trace("block " + block + ": " + res);
}
switch (res) {
case UNDER_REPLICATED:
nrUnderReplicated++;
break;
case OVER_REPLICATED:
nrOverReplicated++;
break;
case INVALID:
nrInvalid++;
break;
case POSTPONE:
nrPostponed++;
postponeBlock(block);
break;
case UNDER_CONSTRUCTION:
nrUnderConstruction++;
break;
case OK:
break;
default:
throw new AssertionError("Invalid enum value: " + res);
}
processed++;
}
totalProcessed += processed;
// there is a possibility that if any of the blocks deleted/added during
// initialisation, then progress might be different.
replicationQueuesInitProgress = Math.min((double) totalProcessed
/ totalBlocks, 1.0);
if (!blocksItr.hasNext()) {
LOG.info("Total number of blocks = " + blocksMap.size());
LOG.info("Number of invalid blocks = " + nrInvalid);
LOG.info("Number of under-replicated blocks = " + nrUnderReplicated);
LOG.info("Number of over-replicated blocks = " + nrOverReplicated
+ ((nrPostponed > 0) ? (" (" + nrPostponed + " postponed)") : ""));
LOG.info("Number of blocks being written = " + nrUnderConstruction);
NameNode.stateChangeLog
.info("STATE* Replication Queue initialization "
+ "scan for invalid, over- and under-replicated blocks "
+ "completed in " + (Time.now() - startTimeMisReplicatedScan)
+ " msec");
break;
}
} finally {
namesystem.writeUnlock();
}
}
if (Thread.currentThread().isInterrupted()) {
LOG.info("Interrupted while processing replication queues.");
}
}
/**
* Get the progress of the Replication queues initialisation
*
* @return Returns values between 0 and 1 for the progress.
*/
public double getReplicationQueuesInitProgress() {
return replicationQueuesInitProgress;
} }
/** /**
@ -3352,6 +3453,7 @@ enum MisReplicationResult {
} }
public void shutdown() { public void shutdown() {
stopReplicationInitializer();
blocksMap.close(); blocksMap.close();
} }
} }

View File

@ -23,6 +23,7 @@
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.GSet; import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet; import org.apache.hadoop.util.LightWeightGSet;
import org.apache.hadoop.util.LightWeightGSet.SetIterator;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
@ -66,7 +67,20 @@ public void remove() {
BlocksMap(int capacity) { BlocksMap(int capacity) {
// Use 2% of total memory to size the GSet capacity // Use 2% of total memory to size the GSet capacity
this.capacity = capacity; this.capacity = capacity;
this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity); this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity) {
@Override
public Iterator<BlockInfo> iterator() {
SetIterator iterator = new SetIterator();
/*
* Not tracking any modifications to set. As this set will be used
* always under FSNameSystem lock, modifications will not cause any
* ConcurrentModificationExceptions. But there is a chance of missing
* newly added elements during iteration.
*/
iterator.setTrackModification(false);
return iterator;
}
};
} }

View File

@ -20,10 +20,14 @@
import java.io.DataInput; import java.io.DataInput;
import java.io.DataOutput; import java.io.DataOutput;
import java.io.IOException; import java.io.IOException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext; import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext;
import com.google.common.base.Preconditions;
/************************************ /************************************
* Some handy internal HDFS constants * Some handy internal HDFS constants
* *
@ -43,6 +47,33 @@ static public enum NodeType {
JOURNAL_NODE; JOURNAL_NODE;
} }
/** Startup options for rolling upgrade. */
public static enum RollingUpgradeStartupOption{
ROLLBACK, DOWNGRADE, STARTED;
public String getOptionString() {
return StartupOption.ROLLINGUPGRADE.getName() + " "
+ name().toLowerCase();
}
public boolean matches(StartupOption option) {
return option == StartupOption.ROLLINGUPGRADE
&& option.getRollingUpgradeStartupOption() == this;
}
private static final RollingUpgradeStartupOption[] VALUES = values();
static RollingUpgradeStartupOption fromString(String s) {
for(RollingUpgradeStartupOption opt : VALUES) {
if (opt.name().equalsIgnoreCase(s)) {
return opt;
}
}
throw new IllegalArgumentException("Failed to convert \"" + s
+ "\" to " + RollingUpgradeStartupOption.class.getSimpleName());
}
}
/** Startup options */ /** Startup options */
static public enum StartupOption{ static public enum StartupOption{
FORMAT ("-format"), FORMAT ("-format"),
@ -54,6 +85,7 @@ static public enum StartupOption{
UPGRADE ("-upgrade"), UPGRADE ("-upgrade"),
ROLLBACK("-rollback"), ROLLBACK("-rollback"),
FINALIZE("-finalize"), FINALIZE("-finalize"),
ROLLINGUPGRADE("-rollingUpgrade"),
IMPORT ("-importCheckpoint"), IMPORT ("-importCheckpoint"),
BOOTSTRAPSTANDBY("-bootstrapStandby"), BOOTSTRAPSTANDBY("-bootstrapStandby"),
INITIALIZESHAREDEDITS("-initializeSharedEdits"), INITIALIZESHAREDEDITS("-initializeSharedEdits"),
@ -61,12 +93,18 @@ static public enum StartupOption{
FORCE("-force"), FORCE("-force"),
NONINTERACTIVE("-nonInteractive"), NONINTERACTIVE("-nonInteractive"),
RENAMERESERVED("-renameReserved"); RENAMERESERVED("-renameReserved");
private static final Pattern ENUM_WITH_ROLLING_UPGRADE_OPTION = Pattern.compile(
"(\\w+)\\((\\w+)\\)");
private final String name; private final String name;
// Used only with format and upgrade options // Used only with format and upgrade options
private String clusterId = null; private String clusterId = null;
// Used only by rolling upgrade
private RollingUpgradeStartupOption rollingUpgradeStartupOption;
// Used only with format option // Used only with format option
private boolean isForceFormat = false; private boolean isForceFormat = false;
private boolean isInteractiveFormat = true; private boolean isInteractiveFormat = true;
@ -94,6 +132,16 @@ public void setClusterId(String cid) {
public String getClusterId() { public String getClusterId() {
return clusterId; return clusterId;
} }
public void setRollingUpgradeStartupOption(String opt) {
Preconditions.checkState(this == ROLLINGUPGRADE);
rollingUpgradeStartupOption = RollingUpgradeStartupOption.fromString(opt);
}
public RollingUpgradeStartupOption getRollingUpgradeStartupOption() {
Preconditions.checkState(this == ROLLINGUPGRADE);
return rollingUpgradeStartupOption;
}
public MetaRecoveryContext createRecoveryContext() { public MetaRecoveryContext createRecoveryContext() {
if (!name.equals(RECOVER.name)) if (!name.equals(RECOVER.name))
@ -124,6 +172,27 @@ public boolean getInteractiveFormat() {
public void setInteractiveFormat(boolean interactive) { public void setInteractiveFormat(boolean interactive) {
isInteractiveFormat = interactive; isInteractiveFormat = interactive;
} }
@Override
public String toString() {
if (this == ROLLINGUPGRADE) {
return new StringBuilder(super.toString())
.append("(").append(getRollingUpgradeStartupOption()).append(")")
.toString();
}
return super.toString();
}
static public StartupOption getEnum(String value) {
Matcher matcher = ENUM_WITH_ROLLING_UPGRADE_OPTION.matcher(value);
if (matcher.matches()) {
StartupOption option = StartupOption.valueOf(matcher.group(1));
option.setRollingUpgradeStartupOption(matcher.group(2));
return option;
} else {
return StartupOption.valueOf(value);
}
}
} }
// Timeouts for communicating with DataNode for streaming writes/reads // Timeouts for communicating with DataNode for streaming writes/reads

View File

@ -45,8 +45,9 @@ public IncorrectVersionException(String minimumVersion, String reportedVersion,
minimumVersion + "'"); minimumVersion + "'");
} }
public IncorrectVersionException(int versionReported, String ofWhat) { public IncorrectVersionException(int currentLayoutVersion,
this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION); int versionReported, String ofWhat) {
this(versionReported, ofWhat, currentLayoutVersion);
} }
public IncorrectVersionException(int versionReported, public IncorrectVersionException(int versionReported,

View File

@ -26,26 +26,22 @@
import java.nio.channels.FileLock; import java.nio.channels.FileLock;
import java.nio.channels.OverlappingFileLockException; import java.nio.channels.OverlappingFileLockException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Properties; import java.util.Properties;
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.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
import com.google.common.base.Preconditions;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
@ -121,7 +117,6 @@ public interface StorageDirType {
public boolean isOfType(StorageDirType type); public boolean isOfType(StorageDirType type);
} }
protected NodeType storageType; // Type of the node using this storage
protected List<StorageDirectory> storageDirs = new ArrayList<StorageDirectory>(); protected List<StorageDirectory> storageDirs = new ArrayList<StorageDirectory>();
private class DirIterator implements Iterator<StorageDirectory> { private class DirIterator implements Iterator<StorageDirectory> {
@ -752,13 +747,11 @@ public boolean isLockSupported() throws IOException {
* Create empty storage info of the specified type * Create empty storage info of the specified type
*/ */
protected Storage(NodeType type) { protected Storage(NodeType type) {
super(); super(type);
this.storageType = type;
} }
protected Storage(NodeType type, StorageInfo storageInfo) { protected Storage(StorageInfo storageInfo) {
super(storageInfo); super(storageInfo);
this.storageType = type;
} }
public int getNumStorageDirs() { public int getNumStorageDirs() {
@ -888,22 +881,6 @@ public interface FormatConfirmable {
public String toString(); public String toString();
} }
/**
* Get common storage fields.
* Should be overloaded if additional fields need to be get.
*
* @param props
* @throws IOException
*/
protected void setFieldsFromProperties(
Properties props, StorageDirectory sd) throws IOException {
setLayoutVersion(props, sd);
setNamespaceID(props, sd);
setStorageType(props, sd);
setcTime(props, sd);
setClusterId(props, layoutVersion, sd);
}
/** /**
* Set common storage fields into the given properties object. * Set common storage fields into the given properties object.
* Should be overloaded if additional fields need to be set. * Should be overloaded if additional fields need to be set.
@ -917,7 +894,7 @@ protected void setPropertiesFromFields(Properties props,
props.setProperty("storageType", storageType.toString()); props.setProperty("storageType", storageType.toString());
props.setProperty("namespaceID", String.valueOf(namespaceID)); props.setProperty("namespaceID", String.valueOf(namespaceID));
// Set clusterID in version with federation support // Set clusterID in version with federation support
if (versionSupportsFederation()) { if (versionSupportsFederation(getServiceLayoutFeatureMap())) {
props.setProperty("clusterID", clusterID); props.setProperty("clusterID", clusterID);
} }
props.setProperty("cTime", String.valueOf(cTime)); props.setProperty("cTime", String.valueOf(cTime));
@ -1016,7 +993,7 @@ public static void deleteDir(File dir) throws IOException {
* @throws IOException * @throws IOException
*/ */
public void writeAll() throws IOException { public void writeAll() throws IOException {
this.layoutVersion = HdfsConstants.LAYOUT_VERSION; this.layoutVersion = getServiceLayoutVersion();
for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) { for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
writeProperties(it.next()); writeProperties(it.next());
} }
@ -1039,73 +1016,9 @@ public static String getBuildVersion() {
public static String getRegistrationID(StorageInfo storage) { public static String getRegistrationID(StorageInfo storage) {
return "NS-" + Integer.toString(storage.getNamespaceID()) return "NS-" + Integer.toString(storage.getNamespaceID())
+ "-" + storage.getClusterID() + "-" + storage.getClusterID()
+ "-" + Integer.toString(storage.getLayoutVersion())
+ "-" + Long.toString(storage.getCTime()); + "-" + Long.toString(storage.getCTime());
} }
String getProperty(Properties props, StorageDirectory sd,
String name) throws InconsistentFSStateException {
String property = props.getProperty(name);
if (property == null) {
throw new InconsistentFSStateException(sd.root, "file "
+ STORAGE_FILE_VERSION + " has " + name + " missing.");
}
return property;
}
/** Validate and set storage type from {@link Properties}*/
protected void setStorageType(Properties props, StorageDirectory sd)
throws InconsistentFSStateException {
NodeType type = NodeType.valueOf(getProperty(props, sd, "storageType"));
if (!storageType.equals(type)) {
throw new InconsistentFSStateException(sd.root,
"node type is incompatible with others.");
}
storageType = type;
}
/** Validate and set ctime from {@link Properties}*/
protected void setcTime(Properties props, StorageDirectory sd)
throws InconsistentFSStateException {
cTime = Long.parseLong(getProperty(props, sd, "cTime"));
}
/** Validate and set clusterId from {@link Properties}*/
protected void setClusterId(Properties props, int layoutVersion,
StorageDirectory sd) throws InconsistentFSStateException {
// Set cluster ID in version that supports federation
if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
String cid = getProperty(props, sd, "clusterID");
if (!(clusterID.equals("") || cid.equals("") || clusterID.equals(cid))) {
throw new InconsistentFSStateException(sd.getRoot(),
"cluster Id is incompatible with others.");
}
clusterID = cid;
}
}
/** Validate and set layout version from {@link Properties}*/
protected void setLayoutVersion(Properties props, StorageDirectory sd)
throws IncorrectVersionException, InconsistentFSStateException {
int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
if (lv < HdfsConstants.LAYOUT_VERSION) { // future version
throw new IncorrectVersionException(lv, "storage directory "
+ sd.root.getAbsolutePath());
}
layoutVersion = lv;
}
/** Validate and set namespaceID version from {@link Properties}*/
protected void setNamespaceID(Properties props, StorageDirectory sd)
throws InconsistentFSStateException {
int nsId = Integer.parseInt(getProperty(props, sd, "namespaceID"));
if (namespaceID != 0 && nsId != 0 && namespaceID != nsId) {
throw new InconsistentFSStateException(sd.root,
"namespaceID is incompatible with others.");
}
namespaceID = nsId;
}
public static boolean is203LayoutVersion(int layoutVersion) { public static boolean is203LayoutVersion(int layoutVersion) {
for (int lv203 : LAYOUT_VERSIONS_203) { for (int lv203 : LAYOUT_VERSIONS_203) {
if (lv203 == layoutVersion) { if (lv203 == layoutVersion) {

View File

@ -17,9 +17,20 @@
*/ */
package org.apache.hadoop.hdfs.server.common; package org.apache.hadoop.hdfs.server.common;
import java.io.IOException;
import java.util.Map;
import java.util.Properties;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
@ -34,20 +45,26 @@ public class StorageInfo {
public int namespaceID; // id of the file system public int namespaceID; // id of the file system
public String clusterID; // id of the cluster public String clusterID; // id of the cluster
public long cTime; // creation time of the file system state public long cTime; // creation time of the file system state
public StorageInfo () { protected final NodeType storageType; // Type of the node using this storage
this(0, 0, "", 0L);
protected static final String STORAGE_FILE_VERSION = "VERSION";
public StorageInfo(NodeType type) {
this(0, 0, "", 0L, type);
} }
public StorageInfo(int layoutV, int nsID, String cid, long cT) { public StorageInfo(int layoutV, int nsID, String cid, long cT, NodeType type) {
layoutVersion = layoutV; layoutVersion = layoutV;
clusterID = cid; clusterID = cid;
namespaceID = nsID; namespaceID = nsID;
cTime = cT; cTime = cT;
storageType = type;
} }
public StorageInfo(StorageInfo from) { public StorageInfo(StorageInfo from) {
setStorageInfo(from); this(from.layoutVersion, from.namespaceID, from.clusterID, from.cTime,
from.storageType);
} }
/** /**
@ -80,8 +97,10 @@ public void setStorageInfo(StorageInfo from) {
cTime = from.cTime; cTime = from.cTime;
} }
public boolean versionSupportsFederation() { public boolean versionSupportsFederation(
return LayoutVersion.supports(Feature.FEDERATION, layoutVersion); Map<Integer, SortedSet<LayoutFeature>> map) {
return LayoutVersion.supports(map, LayoutVersion.Feature.FEDERATION,
layoutVersion);
} }
@Override @Override
@ -96,4 +115,97 @@ public String toColonSeparatedString() {
return Joiner.on(":").join( return Joiner.on(":").join(
layoutVersion, namespaceID, cTime, clusterID); layoutVersion, namespaceID, cTime, clusterID);
} }
/**
* Get common storage fields.
* Should be overloaded if additional fields need to be get.
*
* @param props
* @throws IOException
*/
protected void setFieldsFromProperties(
Properties props, StorageDirectory sd) throws IOException {
setLayoutVersion(props, sd);
setNamespaceID(props, sd);
setcTime(props, sd);
setClusterId(props, layoutVersion, sd);
checkStorageType(props, sd);
}
/** Validate and set storage type from {@link Properties}*/
protected void checkStorageType(Properties props, StorageDirectory sd)
throws InconsistentFSStateException {
if (storageType == null) { //don't care about storage type
return;
}
NodeType type = NodeType.valueOf(getProperty(props, sd, "storageType"));
if (!storageType.equals(type)) {
throw new InconsistentFSStateException(sd.root,
"Incompatible node types: storageType=" + storageType
+ " but StorageDirectory type=" + type);
}
}
/** Validate and set ctime from {@link Properties}*/
protected void setcTime(Properties props, StorageDirectory sd)
throws InconsistentFSStateException {
cTime = Long.parseLong(getProperty(props, sd, "cTime"));
}
/** Validate and set clusterId from {@link Properties}*/
protected void setClusterId(Properties props, int layoutVersion,
StorageDirectory sd) throws InconsistentFSStateException {
// Set cluster ID in version that supports federation
if (LayoutVersion.supports(getServiceLayoutFeatureMap(),
Feature.FEDERATION, layoutVersion)) {
String cid = getProperty(props, sd, "clusterID");
if (!(clusterID.equals("") || cid.equals("") || clusterID.equals(cid))) {
throw new InconsistentFSStateException(sd.getRoot(),
"cluster Id is incompatible with others.");
}
clusterID = cid;
}
}
/** Validate and set layout version from {@link Properties}*/
protected void setLayoutVersion(Properties props, StorageDirectory sd)
throws IncorrectVersionException, InconsistentFSStateException {
int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
if (lv < getServiceLayoutVersion()) { // future version
throw new IncorrectVersionException(getServiceLayoutVersion(), lv,
"storage directory " + sd.root.getAbsolutePath());
}
layoutVersion = lv;
}
/** Validate and set namespaceID version from {@link Properties}*/
protected void setNamespaceID(Properties props, StorageDirectory sd)
throws InconsistentFSStateException {
int nsId = Integer.parseInt(getProperty(props, sd, "namespaceID"));
if (namespaceID != 0 && nsId != 0 && namespaceID != nsId) {
throw new InconsistentFSStateException(sd.root,
"namespaceID is incompatible with others.");
}
namespaceID = nsId;
}
public int getServiceLayoutVersion() {
return storageType == NodeType.DATA_NODE ? HdfsConstants.DATANODE_LAYOUT_VERSION
: HdfsConstants.NAMENODE_LAYOUT_VERSION;
}
public Map<Integer, SortedSet<LayoutFeature>> getServiceLayoutFeatureMap() {
return storageType == NodeType.DATA_NODE? DataNodeLayoutVersion.FEATURES
: NameNodeLayoutVersion.FEATURES;
}
static String getProperty(Properties props, StorageDirectory sd,
String name) throws InconsistentFSStateException {
String property = props.getProperty(name);
if (property == null) {
throw new InconsistentFSStateException(sd.root, "file "
+ STORAGE_FILE_VERSION + " has " + name + " missing.");
}
return property;
}
} }

View File

@ -17,13 +17,10 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import java.io.IOException; import com.google.common.annotations.VisibleForTesting;
import java.net.InetSocketAddress; import com.google.common.base.Preconditions;
import java.util.ArrayList; import com.google.common.collect.Lists;
import java.util.List; import com.google.common.collect.Sets;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@ -32,24 +29,15 @@
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand; import org.apache.hadoop.hdfs.server.protocol.*;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import com.google.common.annotations.VisibleForTesting; import java.io.IOException;
import com.google.common.base.Preconditions; import java.net.InetSocketAddress;
import com.google.common.collect.Lists; import java.util.ArrayList;
import com.google.common.collect.Sets; import java.util.List;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
/** /**
* One instance per block-pool/namespace on the DN, which handles the * One instance per block-pool/namespace on the DN, which handles the
@ -414,6 +402,18 @@ List<BPServiceActor> getBPServiceActors() {
return Lists.newArrayList(bpServices); return Lists.newArrayList(bpServices);
} }
/**
* Signal the current rolling upgrade status as indicated by the NN.
* @param inProgress true if a rolling upgrade is in progress
*/
void signalRollingUpgrade(boolean inProgress) {
if (inProgress) {
dn.getFSDataset().enableTrash(getBlockPoolId());
} else {
dn.getFSDataset().restoreTrash(getBlockPoolId());
}
}
/** /**
* Update the BPOS's view of which NN is active, based on a heartbeat * Update the BPOS's view of which NN is active, based on a heartbeat
* response from one of the actors. * response from one of the actors.
@ -612,7 +612,8 @@ private boolean processCommandFromActive(DatanodeCommand cmd,
// See HDFS-2987. // See HDFS-2987.
throw new UnsupportedOperationException("Received unimplemented DNA_SHUTDOWN"); throw new UnsupportedOperationException("Received unimplemented DNA_SHUTDOWN");
case DatanodeProtocol.DNA_FINALIZE: case DatanodeProtocol.DNA_FINALIZE:
String bp = ((FinalizeCommand) cmd).getBlockPoolId(); String bp = ((FinalizeCommand) cmd).getBlockPoolId();
LOG.info("Got finalize command for block pool " + bp);
assert getBlockPoolId().equals(bp) : assert getBlockPoolId().equals(bp) :
"BP " + getBlockPoolId() + " received DNA_FINALIZE " + "BP " + getBlockPoolId() + " received DNA_FINALIZE " +
"for other block pool " + bp; "for other block pool " + bp;

View File

@ -33,8 +33,8 @@
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
@ -204,14 +204,6 @@ private void checkNNVersion(NamespaceInfo nsInfo)
"DataNode version '" + dnVersion + "' but is within acceptable " + "DataNode version '" + dnVersion + "' but is within acceptable " +
"limits. Note: This is normal during a rolling upgrade."); "limits. Note: This is normal during a rolling upgrade.");
} }
if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
LOG.warn("DataNode and NameNode layout versions must be the same." +
" Expected: "+ HdfsConstants.LAYOUT_VERSION +
" actual "+ nsInfo.getLayoutVersion());
throw new IncorrectVersionException(
nsInfo.getLayoutVersion(), "namenode");
}
} }
private void connectToNNAndHandshake() throws IOException { private void connectToNNAndHandshake() throws IOException {
@ -625,6 +617,20 @@ private synchronized void cleanUp() {
bpos.shutdownActor(this); bpos.shutdownActor(this);
} }
private void handleRollingUpgradeStatus(HeartbeatResponse resp) {
RollingUpgradeStatus rollingUpgradeStatus = resp.getRollingUpdateStatus();
if (rollingUpgradeStatus != null &&
rollingUpgradeStatus.getBlockPoolId().compareTo(bpos.getBlockPoolId()) != 0) {
// Can this ever occur?
LOG.error("Invalid BlockPoolId " +
rollingUpgradeStatus.getBlockPoolId() +
" in HeartbeatResponse. Expected " +
bpos.getBlockPoolId());
} else {
bpos.signalRollingUpgrade(rollingUpgradeStatus != null);
}
}
/** /**
* Main loop for each BP thread. Run until shutdown, * Main loop for each BP thread. Run until shutdown,
* forever calling remote NameNode functions. * forever calling remote NameNode functions.
@ -671,6 +677,10 @@ private void offerService() throws Exception {
this, resp.getNameNodeHaState()); this, resp.getNameNodeHaState());
state = resp.getNameNodeHaState().getState(); state = resp.getNameNodeHaState().getState();
if (state == HAServiceState.ACTIVE) {
handleRollingUpgradeStatus(resp);
}
long startProcessCommands = now(); long startProcessCommands = now();
if (!processCommand(resp.getCommands())) if (!processCommand(resp.getCommands()))
continue; continue;

View File

@ -18,6 +18,20 @@
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.util.Daemon;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -27,20 +41,6 @@
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.util.Daemon;
/** /**
* Manages storage for the set of BlockPoolSlices which share a particular * Manages storage for the set of BlockPoolSlices which share a particular
* block pool id, on this DataNode. * block pool id, on this DataNode.
@ -58,14 +58,24 @@
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class BlockPoolSliceStorage extends Storage { public class BlockPoolSliceStorage extends Storage {
private static final Pattern BLOCK_POOL_PATH_PATTERN = Pattern static final String TRASH_ROOT_DIR = "trash";
.compile("^(.*)"
+ "(\\/BP-[0-9]+\\-\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\-[0-9]+\\/.*)$"); private static final String BLOCK_POOL_ID_PATTERN_BASE =
"/BP-\\d+-\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}-\\d+/";
private static final Pattern BLOCK_POOL_PATH_PATTERN = Pattern.compile(
"^(.*)(" + BLOCK_POOL_ID_PATTERN_BASE + ")(.*)$");
private static final Pattern BLOCK_POOL_CURRENT_PATH_PATTERN = Pattern.compile(
"^(.*)(" + BLOCK_POOL_ID_PATTERN_BASE + ")(" + STORAGE_DIR_CURRENT + ")(.*)$");
private static final Pattern BLOCK_POOL_TRASH_PATH_PATTERN = Pattern.compile(
"^(.*)(" + BLOCK_POOL_ID_PATTERN_BASE + ")(" + TRASH_ROOT_DIR + ")(.*)$");
private String blockpoolID = ""; // id of the blockpool private String blockpoolID = ""; // id of the blockpool
public BlockPoolSliceStorage(StorageInfo storageInfo, String bpid) { public BlockPoolSliceStorage(StorageInfo storageInfo, String bpid) {
super(NodeType.DATA_NODE, storageInfo); super(storageInfo);
blockpoolID = bpid; blockpoolID = bpid;
} }
@ -93,9 +103,7 @@ private BlockPoolSliceStorage() {
*/ */
void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo, void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
Collection<File> dataDirs, StartupOption startOpt) throws IOException { Collection<File> dataDirs, StartupOption startOpt) throws IOException {
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() LOG.info("Analyzing storage directories for bpid " + nsInfo.getBlockPoolID());
: "Block-pool and name-node layout versions must be the same.";
// 1. For each BP data directory analyze the state and // 1. For each BP data directory analyze the state and
// check whether all is consistent before transitioning. // check whether all is consistent before transitioning.
this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size()); this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
@ -143,8 +151,6 @@ void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
// while others could be up-to-date for the regular startup. // while others could be up-to-date for the regular startup.
for (int idx = 0; idx < getNumStorageDirs(); idx++) { for (int idx = 0; idx < getNumStorageDirs(); idx++) {
doTransition(getStorageDir(idx), nsInfo, startOpt); doTransition(getStorageDir(idx), nsInfo, startOpt);
assert getLayoutVersion() == nsInfo.getLayoutVersion()
: "Data-node and name-node layout versions must be the same.";
assert getCTime() == nsInfo.getCTime() assert getCTime() == nsInfo.getCTime()
: "Data-node and name-node CTimes must be the same."; : "Data-node and name-node CTimes must be the same.";
} }
@ -167,7 +173,7 @@ void format(File dnCurDir, NamespaceInfo nsInfo) throws IOException {
/** /**
* Format a block pool slice storage. * Format a block pool slice storage.
* @param sd the block pool storage * @param bpSdir the block pool storage
* @param nsInfo the name space info * @param nsInfo the name space info
* @throws IOException Signals that an I/O exception has occurred. * @throws IOException Signals that an I/O exception has occurred.
*/ */
@ -175,11 +181,10 @@ private void format(StorageDirectory bpSdir, NamespaceInfo nsInfo) throws IOExce
LOG.info("Formatting block pool " + blockpoolID + " directory " LOG.info("Formatting block pool " + blockpoolID + " directory "
+ bpSdir.getCurrentDir()); + bpSdir.getCurrentDir());
bpSdir.clearDirectory(); // create directory bpSdir.clearDirectory(); // create directory
this.layoutVersion = HdfsConstants.LAYOUT_VERSION; this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
this.cTime = nsInfo.getCTime(); this.cTime = nsInfo.getCTime();
this.namespaceID = nsInfo.getNamespaceID(); this.namespaceID = nsInfo.getNamespaceID();
this.blockpoolID = nsInfo.getBlockPoolID(); this.blockpoolID = nsInfo.getBlockPoolID();
this.storageType = NodeType.DATA_NODE;
writeProperties(bpSdir); writeProperties(bpSdir);
} }
@ -206,7 +211,7 @@ private void setBlockPoolID(File storage, String bpid)
if (!blockpoolID.equals("") && !blockpoolID.equals(bpid)) { if (!blockpoolID.equals("") && !blockpoolID.equals(bpid)) {
throw new InconsistentFSStateException(storage, throw new InconsistentFSStateException(storage,
"Unexepcted blockpoolID " + bpid + " . Expected " + blockpoolID); "Unexpected blockpoolID " + bpid + ". Expected " + blockpoolID);
} }
blockpoolID = bpid; blockpoolID = bpid;
} }
@ -230,7 +235,6 @@ protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
* Upgrade if this.LV > LAYOUT_VERSION || this.cTime < namenode.cTime Regular * Upgrade if this.LV > LAYOUT_VERSION || this.cTime < namenode.cTime Regular
* startup if this.LV = LAYOUT_VERSION && this.cTime = namenode.cTime * startup if this.LV = LAYOUT_VERSION && this.cTime = namenode.cTime
* *
* @param dn DataNode to which this storage belongs to
* @param sd storage directory <SD>/current/<bpid> * @param sd storage directory <SD>/current/<bpid>
* @param nsInfo namespace info * @param nsInfo namespace info
* @param startOpt startup option * @param startOpt startup option
@ -238,12 +242,18 @@ protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
*/ */
private void doTransition(StorageDirectory sd, private void doTransition(StorageDirectory sd,
NamespaceInfo nsInfo, StartupOption startOpt) throws IOException { NamespaceInfo nsInfo, StartupOption startOpt) throws IOException {
if (startOpt == StartupOption.ROLLBACK) if (startOpt == StartupOption.ROLLBACK) {
doRollback(sd, nsInfo); // rollback if applicable doRollback(sd, nsInfo); // rollback if applicable
} else {
// Restore all the files in the trash. The restored files are retained
// during rolling upgrade rollback. They are deleted during rolling
// upgrade downgrade.
int restored = restoreBlockFilesFromTrash(getTrashRootDir(sd));
LOG.info("Restored " + restored + " block files from trash.");
}
readProperties(sd); readProperties(sd);
checkVersionUpgradable(this.layoutVersion); checkVersionUpgradable(this.layoutVersion);
assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION
: "Future version is not allowed"; : "Future version is not allowed";
if (getNamespaceID() != nsInfo.getNamespaceID()) { if (getNamespaceID() != nsInfo.getNamespaceID()) {
throw new IOException("Incompatible namespaceIDs in " throw new IOException("Incompatible namespaceIDs in "
@ -257,11 +267,11 @@ private void doTransition(StorageDirectory sd,
+ nsInfo.getBlockPoolID() + "; datanode blockpoolID = " + nsInfo.getBlockPoolID() + "; datanode blockpoolID = "
+ blockpoolID); + blockpoolID);
} }
if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION
&& this.cTime == nsInfo.getCTime()) { && this.cTime == nsInfo.getCTime()) {
return; // regular startup return; // regular startup
} }
if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION
|| this.cTime < nsInfo.getCTime()) { || this.cTime < nsInfo.getCTime()) {
doUpgrade(sd, nsInfo); // upgrade doUpgrade(sd, nsInfo); // upgrade
return; return;
@ -294,7 +304,8 @@ private void doTransition(StorageDirectory sd,
*/ */
void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException { void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException {
// Upgrading is applicable only to release with federation or after // Upgrading is applicable only to release with federation or after
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) { if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
return; return;
} }
LOG.info("Upgrading block pool storage directory " + bpSd.getRoot() LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
@ -322,18 +333,20 @@ void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException {
File bpTmpDir = bpSd.getPreviousTmp(); File bpTmpDir = bpSd.getPreviousTmp();
assert !bpTmpDir.exists() : "previous.tmp directory must not exist."; assert !bpTmpDir.exists() : "previous.tmp directory must not exist.";
// 2. Rename <SD>/curernt/<bpid>/current to <SD>/curernt/<bpid>/previous.tmp // 2. Rename <SD>/current/<bpid>/current to
// <SD>/current/<bpid>/previous.tmp
rename(bpCurDir, bpTmpDir); rename(bpCurDir, bpTmpDir);
// 3. Create new <SD>/current with block files hardlinks and VERSION // 3. Create new <SD>/current with block files hardlinks and VERSION
linkAllBlocks(bpTmpDir, bpCurDir); linkAllBlocks(bpTmpDir, bpCurDir);
this.layoutVersion = HdfsConstants.LAYOUT_VERSION; this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
assert this.namespaceID == nsInfo.getNamespaceID() assert this.namespaceID == nsInfo.getNamespaceID()
: "Data-node and name-node layout versions must be the same."; : "Data-node and name-node layout versions must be the same.";
this.cTime = nsInfo.getCTime(); this.cTime = nsInfo.getCTime();
writeProperties(bpSd); writeProperties(bpSd);
// 4.rename <SD>/curernt/<bpid>/previous.tmp to <SD>/curernt/<bpid>/previous // 4.rename <SD>/current/<bpid>/previous.tmp to
// <SD>/current/<bpid>/previous
rename(bpTmpDir, bpPrevDir); rename(bpTmpDir, bpPrevDir);
LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot() LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot()
+ " is complete"); + " is complete");
@ -349,7 +362,8 @@ void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException {
* @throws IOException if the directory is not empty or it can not be removed * @throws IOException if the directory is not empty or it can not be removed
*/ */
private void cleanupDetachDir(File detachDir) throws IOException { private void cleanupDetachDir(File detachDir) throws IOException {
if (!LayoutVersion.supports(Feature.APPEND_RBW_DIR, layoutVersion) if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.APPEND_RBW_DIR, layoutVersion)
&& detachDir.exists() && detachDir.isDirectory()) { && detachDir.exists() && detachDir.isDirectory()) {
if (FileUtil.list(detachDir).length != 0) { if (FileUtil.list(detachDir).length != 0) {
@ -363,6 +377,43 @@ private void cleanupDetachDir(File detachDir) throws IOException {
} }
} }
/**
* Restore all files from the trash directory to their corresponding
* locations under current/
*/
private int restoreBlockFilesFromTrash(File trashRoot)
throws IOException {
int filesRestored = 0;
File[] children = trashRoot.exists() ? trashRoot.listFiles() : null;
if (children == null) {
return 0;
}
File restoreDirectory = null;
for (File child : children) {
if (child.isDirectory()) {
// Recurse to process subdirectories.
filesRestored += restoreBlockFilesFromTrash(child);
continue;
}
if (restoreDirectory == null) {
restoreDirectory = new File(getRestoreDirectory(child));
if (!restoreDirectory.exists() && !restoreDirectory.mkdirs()) {
throw new IOException("Failed to create directory " + restoreDirectory);
}
}
final File newChild = new File(restoreDirectory, child.getName());
if (!child.renameTo(newChild)) {
throw new IOException("Failed to rename " + child + " to " + newChild);
}
++filesRestored;
}
FileUtil.fullyDelete(trashRoot);
return filesRestored;
}
/* /*
* Roll back to old snapshot at the block pool level * Roll back to old snapshot at the block pool level
* If previous directory exists: * If previous directory exists:
@ -389,13 +440,13 @@ void doRollback(StorageDirectory bpSd, NamespaceInfo nsInfo)
// the namespace state or can be further upgraded to it. // the namespace state or can be further upgraded to it.
// In another word, we can only roll back when ( storedLV >= software LV) // In another word, we can only roll back when ( storedLV >= software LV)
// && ( DN.previousCTime <= NN.ctime) // && ( DN.previousCTime <= NN.ctime)
if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION && if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION &&
prevInfo.getCTime() <= nsInfo.getCTime())) { // cannot rollback prevInfo.getCTime() <= nsInfo.getCTime())) { // cannot rollback
throw new InconsistentFSStateException(bpSd.getRoot(), throw new InconsistentFSStateException(bpSd.getRoot(),
"Cannot rollback to a newer state.\nDatanode previous state: LV = " "Cannot rollback to a newer state.\nDatanode previous state: LV = "
+ prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime() + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
+ " is newer than the namespace state: LV = " + " is newer than the namespace state: LV = "
+ nsInfo.getLayoutVersion() + " CTime = " + nsInfo.getCTime()); + HdfsConstants.DATANODE_LAYOUT_VERSION + " CTime = " + nsInfo.getCTime());
} }
LOG.info("Rolling back storage directory " + bpSd.getRoot() LOG.info("Rolling back storage directory " + bpSd.getRoot()
@ -478,9 +529,6 @@ private void linkAllBlocks(File fromDir, File toDir) throws IOException {
/** /**
* gets the data node storage directory based on block pool storage * gets the data node storage directory based on block pool storage
*
* @param bpRoot
* @return
*/ */
private static String getDataNodeStorageRoot(String bpRoot) { private static String getDataNodeStorageRoot(String bpRoot) {
Matcher matcher = BLOCK_POOL_PATH_PATTERN.matcher(bpRoot); Matcher matcher = BLOCK_POOL_PATH_PATTERN.matcher(bpRoot);
@ -510,4 +558,66 @@ public static File getBpRoot(String bpID, File dnCurDir) {
public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException { public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
return false; return false;
} }
private File getTrashRootDir(StorageDirectory sd) {
return new File(sd.getRoot(), TRASH_ROOT_DIR);
}
/**
* Get a target subdirectory under trash/ for a given block file that is being
* deleted.
*
* The subdirectory structure under trash/ mirrors that under current/ to keep
* implicit memory of where the files are to be restored (if necessary).
*
* @return the trash directory for a given block file that is being deleted.
*/
public String getTrashDirectory(File blockFile) {
Matcher matcher = BLOCK_POOL_CURRENT_PATH_PATTERN.matcher(blockFile.getParent());
String trashDirectory = matcher.replaceFirst("$1$2" + TRASH_ROOT_DIR + "$4");
return trashDirectory;
}
/**
* Get a target subdirectory under current/ for a given block file that is being
* restored from trash.
*
* The subdirectory structure under trash/ mirrors that under current/ to keep
* implicit memory of where the files are to be restored.
*
* @return the target directory to restore a previously deleted block file.
*/
@VisibleForTesting
String getRestoreDirectory(File blockFile) {
Matcher matcher = BLOCK_POOL_TRASH_PATH_PATTERN.matcher(blockFile.getParent());
String restoreDirectory = matcher.replaceFirst("$1$2" + STORAGE_DIR_CURRENT + "$4");
LOG.info("Restoring " + blockFile + " to " + restoreDirectory);
return restoreDirectory;
}
/**
* Delete all files and directories in the trash directories.
*/
public void restoreTrash() {
for (StorageDirectory sd : storageDirs) {
File trashRoot = getTrashRootDir(sd);
try {
restoreBlockFilesFromTrash(trashRoot);
FileUtil.fullyDelete(getTrashRootDir(sd));
} catch (IOException ioe) {
LOG.warn("Restoring trash failed for storage directory " + sd);
}
}
}
/** trash is enabled if at least one storage directory contains trash root */
@VisibleForTesting
public boolean trashEnabled() {
for (StorageDirectory sd : storageDirs) {
if (getTrashRootDir(sd).exists()) {
return true;
}
}
return false;
}
} }

View File

@ -23,8 +23,10 @@
import java.io.Closeable; import java.io.Closeable;
import java.io.DataInputStream; import java.io.DataInputStream;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.File;
import java.io.FileDescriptor; import java.io.FileDescriptor;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -45,6 +47,7 @@
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams; import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams; import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
@ -52,6 +55,7 @@
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -116,6 +120,7 @@ class BlockReceiver implements Closeable {
private final boolean isTransfer; private final boolean isTransfer;
private boolean syncOnClose; private boolean syncOnClose;
private long restartBudget;
BlockReceiver(final ExtendedBlock block, final DataInputStream in, BlockReceiver(final ExtendedBlock block, final DataInputStream in,
final String inAddr, final String myAddr, final String inAddr, final String myAddr,
@ -135,6 +140,7 @@ class BlockReceiver implements Closeable {
this.clientname = clientname; this.clientname = clientname;
this.isDatanode = clientname.length() == 0; this.isDatanode = clientname.length() == 0;
this.isClient = !this.isDatanode; this.isClient = !this.isDatanode;
this.restartBudget = datanode.getDnConf().restartReplicaExpiry;
//for datanode, we have //for datanode, we have
//1: clientName.length() == 0, and //1: clientName.length() == 0, and
@ -723,19 +729,71 @@ void receiveBlock(
} }
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.info("Exception for " + block, ioe); if (datanode.isRestarting()) {
throw ioe; // Do not throw if shutting down for restart. Otherwise, it will cause
// premature termination of responder.
LOG.info("Shutting down for restart (" + block + ").");
} else {
LOG.info("Exception for " + block, ioe);
throw ioe;
}
} finally { } finally {
if (!responderClosed) { // Abnormal termination of the flow above // Clear the previous interrupt state of this thread.
IOUtils.closeStream(this); Thread.interrupted();
// If a shutdown for restart was initiated, upstream needs to be notified.
// There is no need to do anything special if the responder was closed
// normally.
if (!responderClosed) { // Data transfer was not complete.
if (responder != null) { if (responder != null) {
// In case this datanode is shutting down for quick restart,
// send a special ack upstream.
if (datanode.isRestarting() && isClient && !isTransfer) {
File blockFile = ((ReplicaInPipeline)replicaInfo).getBlockFile();
File restartMeta = new File(blockFile.getParent() +
File.pathSeparator + "." + blockFile.getName() + ".restart");
if (restartMeta.exists() && !restartMeta.delete()) {
LOG.warn("Failed to delete restart meta file: " +
restartMeta.getPath());
}
try {
FileWriter out = new FileWriter(restartMeta);
// write out the current time.
out.write(Long.toString(Time.now() + restartBudget));
out.flush();
out.close();
} catch (IOException ioe) {
// The worst case is not recovering this RBW replica.
// Client will fall back to regular pipeline recovery.
}
try {
((PacketResponder) responder.getRunnable()).
sendOOBResponse(PipelineAck.getRestartOOBStatus());
// Even if the connection is closed after the ack packet is
// flushed, the client can react to the connection closure
// first. Insert a delay to lower the chance of client
// missing the OOB ack.
Thread.sleep(1000);
} catch (InterruptedException ie) {
// It is already going down. Ignore this.
} catch (IOException ioe) {
LOG.info("Error sending OOB Ack.", ioe);
}
}
responder.interrupt(); responder.interrupt();
} }
IOUtils.closeStream(this);
cleanupBlock(); cleanupBlock();
} }
if (responder != null) { if (responder != null) {
try { try {
responder.join(datanode.getDnConf().getXceiverStopTimeout()); responder.interrupt();
// join() on the responder should timeout a bit earlier than the
// configured deadline. Otherwise, the join() on this thread will
// likely timeout as well.
long joinTimeout = datanode.getDnConf().getXceiverStopTimeout();
joinTimeout = joinTimeout > 1 ? joinTimeout*8/10 : joinTimeout;
responder.join(joinTimeout);
if (responder.isAlive()) { if (responder.isAlive()) {
String msg = "Join on responder thread " + responder String msg = "Join on responder thread " + responder
+ " timed out"; + " timed out";
@ -744,7 +802,10 @@ void receiveBlock(
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
responder.interrupt(); responder.interrupt();
throw new IOException("Interrupted receiveBlock"); // do not throw if shutting down for restart.
if (!datanode.isRestarting()) {
throw new IOException("Interrupted receiveBlock");
}
} }
responder = null; responder = null;
} }
@ -862,6 +923,7 @@ class PacketResponder implements Runnable, Closeable {
private final PacketResponderType type; private final PacketResponderType type;
/** for log and error messages */ /** for log and error messages */
private final String myString; private final String myString;
private boolean sending = false;
@Override @Override
public String toString() { public String toString() {
@ -887,7 +949,9 @@ public String toString() {
} }
private boolean isRunning() { private boolean isRunning() {
return running && datanode.shouldRun; // When preparing for a restart, it should continue to run until
// interrupted by the receiver thread.
return running && (datanode.shouldRun || datanode.isRestarting());
} }
/** /**
@ -903,44 +967,97 @@ void enqueue(final long seqno, final boolean lastPacketInBlock,
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug(myString + ": enqueue " + p); LOG.debug(myString + ": enqueue " + p);
} }
synchronized(this) { synchronized(ackQueue) {
if (running) { if (running) {
ackQueue.addLast(p); ackQueue.addLast(p);
notifyAll(); ackQueue.notifyAll();
}
}
}
/**
* Send an OOB response. If all acks have been sent already for the block
* and the responder is about to close, the delivery is not guaranteed.
* This is because the other end can close the connection independently.
* An OOB coming from downstream will be automatically relayed upstream
* by the responder. This method is used only by originating datanode.
*
* @param ackStatus the type of ack to be sent
*/
void sendOOBResponse(final Status ackStatus) throws IOException,
InterruptedException {
if (!running) {
LOG.info("Cannot send OOB response " + ackStatus +
". Responder not running.");
return;
}
synchronized(this) {
if (sending) {
wait(PipelineAck.getOOBTimeout(ackStatus));
// Didn't get my turn in time. Give up.
if (sending) {
throw new IOException("Could not send OOB reponse in time: "
+ ackStatus);
}
}
sending = true;
}
LOG.info("Sending an out of band ack of type " + ackStatus);
try {
sendAckUpstreamUnprotected(null, PipelineAck.UNKOWN_SEQNO, 0L, 0L,
ackStatus);
} finally {
// Let others send ack. Unless there are miltiple OOB send
// calls, there can be only one waiter, the responder thread.
// In any case, only one needs to be notified.
synchronized(this) {
sending = false;
notify();
} }
} }
} }
/** Wait for a packet with given {@code seqno} to be enqueued to ackQueue */ /** Wait for a packet with given {@code seqno} to be enqueued to ackQueue */
synchronized Packet waitForAckHead(long seqno) throws InterruptedException { Packet waitForAckHead(long seqno) throws InterruptedException {
while (isRunning() && ackQueue.size() == 0) { synchronized(ackQueue) {
if (LOG.isDebugEnabled()) { while (isRunning() && ackQueue.size() == 0) {
LOG.debug(myString + ": seqno=" + seqno + if (LOG.isDebugEnabled()) {
" waiting for local datanode to finish write."); LOG.debug(myString + ": seqno=" + seqno +
" waiting for local datanode to finish write.");
}
ackQueue.wait();
} }
wait(); return isRunning() ? ackQueue.getFirst() : null;
} }
return isRunning() ? ackQueue.getFirst() : null;
} }
/** /**
* wait for all pending packets to be acked. Then shutdown thread. * wait for all pending packets to be acked. Then shutdown thread.
*/ */
@Override @Override
public synchronized void close() { public void close() {
while (isRunning() && ackQueue.size() != 0) { synchronized(ackQueue) {
try { while (isRunning() && ackQueue.size() != 0) {
wait(); try {
} catch (InterruptedException e) { ackQueue.wait();
running = false; } catch (InterruptedException e) {
Thread.currentThread().interrupt(); running = false;
Thread.currentThread().interrupt();
}
} }
if(LOG.isDebugEnabled()) {
LOG.debug(myString + ": closing");
}
running = false;
ackQueue.notifyAll();
} }
if(LOG.isDebugEnabled()) {
LOG.debug(myString + ": closing"); synchronized(this) {
running = false;
notifyAll();
} }
running = false;
notifyAll();
} }
/** /**
@ -968,6 +1085,14 @@ public void run() {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(myString + " got " + ack); LOG.debug(myString + " got " + ack);
} }
// Process an OOB ACK.
Status oobStatus = ack.getOOBStatus();
if (oobStatus != null) {
LOG.info("Relaying an out of band ack of type " + oobStatus);
sendAckUpstream(ack, PipelineAck.UNKOWN_SEQNO, 0L, 0L,
Status.SUCCESS);
continue;
}
seqno = ack.getSeqno(); seqno = ack.getSeqno();
} }
if (seqno != PipelineAck.UNKOWN_SEQNO if (seqno != PipelineAck.UNKOWN_SEQNO
@ -1025,6 +1150,9 @@ public void run() {
* status back to the client because this datanode has a problem. * status back to the client because this datanode has a problem.
* The upstream datanode will detect that this datanode is bad, and * The upstream datanode will detect that this datanode is bad, and
* rightly so. * rightly so.
*
* The receiver thread can also interrupt this thread for sending
* an out-of-band response upstream.
*/ */
LOG.info(myString + ": Thread is interrupted."); LOG.info(myString + ": Thread is interrupted.");
running = false; running = false;
@ -1094,17 +1222,64 @@ private void finalizeBlock(long startTime) throws IOException {
} }
/** /**
* The wrapper for the unprotected version. This is only called by
* the responder's run() method.
*
* @param ack Ack received from downstream * @param ack Ack received from downstream
* @param seqno sequence number of ack to be sent upstream * @param seqno sequence number of ack to be sent upstream
* @param totalAckTimeNanos total ack time including all the downstream * @param totalAckTimeNanos total ack time including all the downstream
* nodes * nodes
* @param offsetInBlock offset in block for the data in packet * @param offsetInBlock offset in block for the data in packet
* @param myStatus the local ack status
*/ */
private void sendAckUpstream(PipelineAck ack, long seqno, private void sendAckUpstream(PipelineAck ack, long seqno,
long totalAckTimeNanos, long offsetInBlock, long totalAckTimeNanos, long offsetInBlock,
Status myStatus) throws IOException { Status myStatus) throws IOException {
try {
// Wait for other sender to finish. Unless there is an OOB being sent,
// the responder won't have to wait.
synchronized(this) {
while(sending) {
wait();
}
sending = true;
}
try {
if (!running) return;
sendAckUpstreamUnprotected(ack, seqno, totalAckTimeNanos,
offsetInBlock, myStatus);
} finally {
synchronized(this) {
sending = false;
notify();
}
}
} catch (InterruptedException ie) {
// The responder was interrupted. Make it go down without
// interrupting the receiver(writer) thread.
running = false;
}
}
/**
* @param ack Ack received from downstream
* @param seqno sequence number of ack to be sent upstream
* @param totalAckTimeNanos total ack time including all the downstream
* nodes
* @param offsetInBlock offset in block for the data in packet
* @param myStatus the local ack status
*/
private void sendAckUpstreamUnprotected(PipelineAck ack, long seqno,
long totalAckTimeNanos, long offsetInBlock, Status myStatus)
throws IOException {
Status[] replies = null; Status[] replies = null;
if (mirrorError) { // ack read error if (ack == null) {
// A new OOB response is being sent from this node. Regardless of
// downstream nodes, reply should contain one reply.
replies = new Status[1];
replies[0] = myStatus;
} else if (mirrorError) { // ack read error
replies = MIRROR_ERROR_STATUS; replies = MIRROR_ERROR_STATUS;
} else { } else {
short ackLen = type == PacketResponderType.LAST_IN_PIPELINE ? 0 : ack short ackLen = type == PacketResponderType.LAST_IN_PIPELINE ? 0 : ack
@ -1130,7 +1305,6 @@ private void sendAckUpstream(PipelineAck ack, long seqno,
&& offsetInBlock > replicaInfo.getBytesAcked()) { && offsetInBlock > replicaInfo.getBytesAcked()) {
replicaInfo.setBytesAcked(offsetInBlock); replicaInfo.setBytesAcked(offsetInBlock);
} }
// send my ack back to upstream datanode // send my ack back to upstream datanode
replyAck.write(upstreamOut); replyAck.write(upstreamOut);
upstreamOut.flush(); upstreamOut.flush();
@ -1152,9 +1326,11 @@ private void sendAckUpstream(PipelineAck ack, long seqno,
* *
* This should be called only when the ack queue is not empty * This should be called only when the ack queue is not empty
*/ */
private synchronized void removeAckHead() { private void removeAckHead() {
ackQueue.removeFirst(); synchronized(ackQueue) {
notifyAll(); ackQueue.removeFirst();
ackQueue.notifyAll();
}
} }
} }

View File

@ -46,6 +46,8 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -82,6 +84,7 @@ public class DNConf {
final String encryptionAlgorithm; final String encryptionAlgorithm;
final long xceiverStopTimeout; final long xceiverStopTimeout;
final long restartReplicaExpiry;
final long maxLockedMemory; final long maxLockedMemory;
@ -157,6 +160,10 @@ public DNConf(Configuration conf) {
this.maxLockedMemory = conf.getLong( this.maxLockedMemory = conf.getLong(
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT); DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);
this.restartReplicaExpiry = conf.getLong(
DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY,
DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT) * 1000L;
} }
// We get minimumNameNodeVersion via a method so it can be mocked out in tests. // We get minimumNameNodeVersion via a method so it can be mocked out in tests.

View File

@ -17,41 +17,10 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*; import com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.util.ExitUtil.terminate; import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import java.io.BufferedOutputStream; import com.google.protobuf.BlockingService;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.net.UnknownHostException;
import java.nio.channels.ClosedByInterruptException;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.SocketChannel;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import javax.management.ObjectName;
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.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -69,37 +38,15 @@
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.net.DomainPeerServer; import org.apache.hadoop.hdfs.net.DomainPeerServer;
import org.apache.hadoop.hdfs.net.TcpPeerServer; import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.*;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService; import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB; import org.apache.hadoop.hdfs.protocolPB.*;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB; import org.apache.hadoop.hdfs.security.token.block.*;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@ -113,11 +60,7 @@
import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets; import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
import org.apache.hadoop.hdfs.server.namenode.StreamFile; import org.apache.hadoop.hdfs.server.namenode.StreamFile;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.*;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.Param; import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.http.HttpConfig;
@ -140,21 +83,24 @@
import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.*;
import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException; import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON; import org.mortbay.util.ajax.JSON;
import com.google.common.annotations.VisibleForTesting; import javax.management.ObjectName;
import com.google.common.base.Joiner; import java.io.*;
import com.google.common.base.Preconditions; import java.lang.management.ManagementFactory;
import com.google.protobuf.BlockingService; import java.net.*;
import java.nio.channels.ClosedByInterruptException;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.SocketChannel;
import java.security.PrivilegedExceptionAction;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
/********************************************************** /**********************************************************
* DataNode is a class (and program) that stores a set of * DataNode is a class (and program) that stores a set of
@ -211,7 +157,14 @@ public class DataNode extends Configured
static final Log ClientTraceLog = static final Log ClientTraceLog =
LogFactory.getLog(DataNode.class.getName() + ".clienttrace"); LogFactory.getLog(DataNode.class.getName() + ".clienttrace");
private static final String USAGE = "Usage: java DataNode [-rollback | -regular]"; private static final String USAGE =
"Usage: java DataNode [-regular | -rollback | -rollingupgrade rollback]\n" +
" -regular : Normal DataNode startup (default).\n" +
" -rollback : Rollback a standard upgrade.\n" +
" -rollingupgrade rollback : Rollback a rolling upgrade operation.\n" +
" Refer to HDFS documentation for the difference between standard\n" +
" and rolling upgrades.";
static final int CURRENT_BLOCK_FORMAT_VERSION = 1; static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
/** /**
@ -223,6 +176,8 @@ public static InetSocketAddress createSocketAddr(String target) {
} }
volatile boolean shouldRun = true; volatile boolean shouldRun = true;
volatile boolean shutdownForUpgrade = false;
private boolean shutdownInProgress = false;
private BlockPoolManager blockPoolManager; private BlockPoolManager blockPoolManager;
volatile FsDatasetSpi<? extends FsVolumeSpi> data = null; volatile FsDatasetSpi<? extends FsVolumeSpi> data = null;
private String clusterId = null; private String clusterId = null;
@ -266,6 +221,7 @@ public static InetSocketAddress createSocketAddr(String target) {
private SecureResources secureResources = null; private SecureResources secureResources = null;
private List<StorageLocation> dataDirs; private List<StorageLocation> dataDirs;
private Configuration conf; private Configuration conf;
private String confVersion;
private final long maxNumberOfBlocksToLog; private final long maxNumberOfBlocksToLog;
private final List<String> usersWithLocalPathAccess; private final List<String> usersWithLocalPathAccess;
@ -294,6 +250,11 @@ public static InetSocketAddress createSocketAddr(String target) {
DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT); DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
confVersion = "core-" +
conf.get("hadoop.common.configuration.version", "UNSPECIFIED") +
",hdfs-" +
conf.get("hadoop.hdfs.configuration.version", "UNSPECIFIED");
// Determine whether we should try to pass file descriptors to clients. // Determine whether we should try to pass file descriptors to clients.
if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT)) { DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT)) {
@ -1235,9 +1196,31 @@ public void shutdown() {
// offerServices may be modified. // offerServices may be modified.
BPOfferService[] bposArray = this.blockPoolManager == null ? null BPOfferService[] bposArray = this.blockPoolManager == null ? null
: this.blockPoolManager.getAllNamenodeThreads(); : this.blockPoolManager.getAllNamenodeThreads();
this.shouldRun = false; // If shutdown is not for restart, set shouldRun to false early.
if (!shutdownForUpgrade) {
shouldRun = false;
}
// When shutting down for restart, DataXceiverServer is interrupted
// in order to avoid any further acceptance of requests, but the peers
// for block writes are not closed until the clients are notified.
if (dataXceiverServer != null) {
((DataXceiverServer) this.dataXceiverServer.getRunnable()).kill();
this.dataXceiverServer.interrupt();
}
// Record the time of initial notification
long timeNotified = Time.now();
if (localDataXceiverServer != null) {
((DataXceiverServer) this.localDataXceiverServer.getRunnable()).kill();
this.localDataXceiverServer.interrupt();
}
// Terminate directory scanner and block scanner
shutdownPeriodicScanners(); shutdownPeriodicScanners();
// Stop the web server
if (infoServer != null) { if (infoServer != null) {
try { try {
infoServer.stop(); infoServer.stop();
@ -1245,26 +1228,24 @@ public void shutdown() {
LOG.warn("Exception shutting down DataNode", e); LOG.warn("Exception shutting down DataNode", e);
} }
} }
if (ipcServer != null) {
ipcServer.stop();
}
if (pauseMonitor != null) { if (pauseMonitor != null) {
pauseMonitor.stop(); pauseMonitor.stop();
} }
// shouldRun is set to false here to prevent certain threads from exiting
// before the restart prep is done.
this.shouldRun = false;
if (dataXceiverServer != null) {
((DataXceiverServer) this.dataXceiverServer.getRunnable()).kill();
this.dataXceiverServer.interrupt();
}
if (localDataXceiverServer != null) {
((DataXceiverServer) this.localDataXceiverServer.getRunnable()).kill();
this.localDataXceiverServer.interrupt();
}
// wait for all data receiver threads to exit // wait for all data receiver threads to exit
if (this.threadGroup != null) { if (this.threadGroup != null) {
int sleepMs = 2; int sleepMs = 2;
while (true) { while (true) {
this.threadGroup.interrupt(); // When shutting down for restart, wait 2.5 seconds before forcing
// termination of receiver threads.
if (!this.shutdownForUpgrade ||
(this.shutdownForUpgrade && (Time.now() - timeNotified > 2500))) {
this.threadGroup.interrupt();
}
LOG.info("Waiting for threadgroup to exit, active threads is " + LOG.info("Waiting for threadgroup to exit, active threads is " +
this.threadGroup.activeCount()); this.threadGroup.activeCount());
if (this.threadGroup.activeCount() == 0) { if (this.threadGroup.activeCount() == 0) {
@ -1294,7 +1275,13 @@ public void shutdown() {
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
} }
} }
// IPC server needs to be shutdown late in the process, otherwise
// shutdown command response won't get sent.
if (ipcServer != null) {
ipcServer.stop();
}
if(blockPoolManager != null) { if(blockPoolManager != null) {
try { try {
this.blockPoolManager.shutDownAll(bposArray); this.blockPoolManager.shutDownAll(bposArray);
@ -1321,6 +1308,13 @@ public void shutdown() {
dataNodeInfoBeanName = null; dataNodeInfoBeanName = null;
} }
if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown(); if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
LOG.info("Shutdown complete.");
synchronized(this) {
// it is already false, but setting it again to avoid a findbug warning.
this.shouldRun = false;
// Notify the main thread.
notifyAll();
}
} }
@ -1793,6 +1787,7 @@ public static List<StorageLocation> getStorageLocations(Configuration conf) {
/** Instantiate & Start a single datanode daemon and wait for it to finish. /** Instantiate & Start a single datanode daemon and wait for it to finish.
* If this thread is specifically interrupted, it will stop waiting. * If this thread is specifically interrupted, it will stop waiting.
*/ */
@VisibleForTesting
public static DataNode createDataNode(String args[], public static DataNode createDataNode(String args[],
Configuration conf) throws IOException { Configuration conf) throws IOException {
return createDataNode(args, conf, null); return createDataNode(args, conf, null);
@ -1801,6 +1796,7 @@ public static DataNode createDataNode(String args[],
/** Instantiate & Start a single datanode daemon and wait for it to finish. /** Instantiate & Start a single datanode daemon and wait for it to finish.
* If this thread is specifically interrupted, it will stop waiting. * If this thread is specifically interrupted, it will stop waiting.
*/ */
@VisibleForTesting
@InterfaceAudience.Private @InterfaceAudience.Private
public static DataNode createDataNode(String args[], Configuration conf, public static DataNode createDataNode(String args[], Configuration conf,
SecureResources resources) throws IOException { SecureResources resources) throws IOException {
@ -1819,7 +1815,11 @@ void join() {
&& blockPoolManager.getAllNamenodeThreads().length == 0) { && blockPoolManager.getAllNamenodeThreads().length == 0) {
shouldRun = false; shouldRun = false;
} }
Thread.sleep(2000); // Terminate if shutdown is complete or 2 seconds after all BPs
// are shutdown.
synchronized(this) {
wait(2000);
}
} catch (InterruptedException ex) { } catch (InterruptedException ex) {
LOG.warn("Received exception in Datanode#join: " + ex); LOG.warn("Received exception in Datanode#join: " + ex);
} }
@ -1911,25 +1911,28 @@ private static void printUsage(PrintStream out) {
* *
* @return false if passed argements are incorrect * @return false if passed argements are incorrect
*/ */
private static boolean parseArguments(String args[], @VisibleForTesting
Configuration conf) { static boolean parseArguments(String args[], Configuration conf) {
int argsLen = (args == null) ? 0 : args.length;
StartupOption startOpt = StartupOption.REGULAR; StartupOption startOpt = StartupOption.REGULAR;
for(int i=0; i < argsLen; i++) { int i = 0;
String cmd = args[i];
if (args != null && args.length != 0) {
String cmd = args[i++];
if ("-r".equalsIgnoreCase(cmd) || "--rack".equalsIgnoreCase(cmd)) { if ("-r".equalsIgnoreCase(cmd) || "--rack".equalsIgnoreCase(cmd)) {
LOG.error("-r, --rack arguments are not supported anymore. RackID " + LOG.error("-r, --rack arguments are not supported anymore. RackID " +
"resolution is handled by the NameNode."); "resolution is handled by the NameNode.");
terminate(1);
} else if ("-rollback".equalsIgnoreCase(cmd)) {
startOpt = StartupOption.ROLLBACK;
} else if ("-regular".equalsIgnoreCase(cmd)) {
startOpt = StartupOption.REGULAR;
} else
return false; return false;
} else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.ROLLBACK;
} else if (StartupOption.REGULAR.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.REGULAR;
} else {
return false;
}
} }
setStartupOption(conf, startOpt); setStartupOption(conf, startOpt);
return true; return (args == null || i == args.length); // Fail if more than one cmd specified!
} }
private static void setStartupOption(Configuration conf, StartupOption opt) { private static void setStartupOption(Configuration conf, StartupOption opt) {
@ -1937,8 +1940,9 @@ private static void setStartupOption(Configuration conf, StartupOption opt) {
} }
static StartupOption getStartupOption(Configuration conf) { static StartupOption getStartupOption(Configuration conf) {
return StartupOption.valueOf(conf.get(DFS_DATANODE_STARTUP_KEY, String value = conf.get(DFS_DATANODE_STARTUP_KEY,
StartupOption.REGULAR.toString())); StartupOption.REGULAR.toString());
return StartupOption.getEnum(value);
} }
/** /**
@ -1969,11 +1973,15 @@ public DataBlockScanner getBlockScanner() {
public static void secureMain(String args[], SecureResources resources) { public static void secureMain(String args[], SecureResources resources) {
int errorCode = 0;
try { try {
StringUtils.startupShutdownMessage(DataNode.class, args, LOG); StringUtils.startupShutdownMessage(DataNode.class, args, LOG);
DataNode datanode = createDataNode(args, null, resources); DataNode datanode = createDataNode(args, null, resources);
if (datanode != null) if (datanode != null) {
datanode.join(); datanode.join();
} else {
errorCode = 1;
}
} catch (Throwable e) { } catch (Throwable e) {
LOG.fatal("Exception in secureMain", e); LOG.fatal("Exception in secureMain", e);
terminate(1, e); terminate(1, e);
@ -1983,7 +1991,7 @@ public static void secureMain(String args[], SecureResources resources) {
// condition was not met. Also, In secure mode, control will go to Jsvc // condition was not met. Also, In secure mode, control will go to Jsvc
// and Datanode process hangs if it does not exit. // and Datanode process hangs if it does not exit.
LOG.warn("Exiting Datanode"); LOG.warn("Exiting Datanode");
terminate(0); terminate(errorCode);
} }
} }
@ -2447,6 +2455,43 @@ public void deleteBlockPool(String blockPoolId, boolean force)
data.deleteBlockPool(blockPoolId, force); data.deleteBlockPool(blockPoolId, force);
} }
@Override // ClientDatanodeProtocol
public synchronized void shutdownDatanode(boolean forUpgrade) throws IOException {
LOG.info("shutdownDatanode command received (upgrade=" + forUpgrade +
"). Shutting down Datanode...");
// Shutdown can be called only once.
if (shutdownInProgress) {
throw new IOException("Shutdown already in progress.");
}
shutdownInProgress = true;
shutdownForUpgrade = forUpgrade;
// Asynchronously start the shutdown process so that the rpc response can be
// sent back.
Thread shutdownThread = new Thread() {
@Override public void run() {
if (!shutdownForUpgrade) {
// Delay the shutdown a bit if not doing for restart.
try {
Thread.sleep(1000);
} catch (InterruptedException ie) { }
}
shutdown();
}
};
shutdownThread.setDaemon(true);
shutdownThread.start();
}
@Override //ClientDatanodeProtocol
public DatanodeLocalInfo getDatanodeInfo() {
long uptime = ManagementFactory.getRuntimeMXBean().getUptime()/1000;
return new DatanodeLocalInfo(VersionInfo.getVersion(),
confVersion, uptime);
}
/** /**
* @param addr rpc address of the namenode * @param addr rpc address of the namenode
* @return true if the datanode is connected to a NameNode at the * @return true if the datanode is connected to a NameNode at the
@ -2472,6 +2517,10 @@ public boolean isBPServiceAlive(String bpid) {
return bp != null ? bp.isAlive() : false; return bp != null ? bp.isAlive() : false;
} }
boolean isRestarting() {
return shutdownForUpgrade;
}
/** /**
* A datanode is considered to be fully started if all the BP threads are * A datanode is considered to be fully started if all the BP threads are
* alive and all the block pools are initialized. * alive and all the block pools are initialized.
@ -2520,6 +2569,11 @@ boolean shouldRun() {
return shouldRun; return shouldRun;
} }
@VisibleForTesting
DataStorage getStorage() {
return storage;
}
public ShortCircuitRegistry getShortCircuitRegistry() { public ShortCircuitRegistry getShortCircuitRegistry() {
return shortCircuitRegistry; return shortCircuitRegistry;
} }

View File

@ -0,0 +1,97 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode;
import java.util.HashMap;
import java.util.Map;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.FeatureInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
@InterfaceAudience.Private
public class DataNodeLayoutVersion {
/** Build layout version and corresponding feature matrix */
public final static Map<Integer, SortedSet<LayoutFeature>> FEATURES =
new HashMap<Integer, SortedSet<LayoutFeature>>();
public static final int CURRENT_LAYOUT_VERSION
= LayoutVersion.getCurrentLayoutVersion(Feature.values());
static{
LayoutVersion.updateMap(FEATURES, LayoutVersion.Feature.values());
LayoutVersion.updateMap(FEATURES, DataNodeLayoutVersion.Feature.values());
}
public static SortedSet<LayoutFeature> getFeatures(int lv) {
return FEATURES.get(lv);
}
public static boolean supports(final LayoutFeature f, final int lv) {
return LayoutVersion.supports(FEATURES, f, lv);
}
/**
* Enums for features that change the layout version.
* <br><br>
* To add a new layout version:
* <ul>
* <li>Define a new enum constant with a short enum name, the new layout version
* and description of the added feature.</li>
* <li>When adding a layout version with an ancestor that is not same as
* its immediate predecessor, use the constructor where a specific ancestor
* can be passed.
* </li>
* </ul>
*/
public static enum Feature implements LayoutFeature {
FIRST_LAYOUT(-55, -53, "First datenode layout", false);
private final FeatureInfo info;
/**
* DataNodeFeature that is added at layout version {@code lv} - 1.
* @param lv new layout version with the addition of this feature
* @param description description of the feature
*/
Feature(final int lv, final String description) {
this(lv, lv + 1, description, false);
}
/**
* DataNode feature that is added at layout version {@code ancestoryLV}.
* @param lv new layout version with the addition of this feature
* @param ancestorLV layout version from which the new lv is derived from.
* @param description description of the feature
* @param reserved true when this is a layout version reserved for previous
* version
* @param features set of features that are to be enabled for this version
*/
Feature(final int lv, final int ancestorLV, final String description,
boolean reserved, Feature... features) {
info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
}
@Override
public FeatureInfo getInfo() {
return info;
}
}
}

View File

@ -18,27 +18,15 @@
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileLock;
import java.util.*;
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.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException; import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@ -50,6 +38,11 @@
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DiskChecker; import org.apache.hadoop.util.DiskChecker;
import java.io.*;
import java.nio.channels.FileLock;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/** /**
* Data storage information file. * Data storage information file.
* <p> * <p>
@ -66,6 +59,13 @@ public class DataStorage extends Storage {
public final static String STORAGE_DIR_FINALIZED = "finalized"; public final static String STORAGE_DIR_FINALIZED = "finalized";
public final static String STORAGE_DIR_TMP = "tmp"; public final static String STORAGE_DIR_TMP = "tmp";
// Set of bpids for which 'trash' is currently enabled.
// When trash is enabled block files are moved under a separate
// 'trash' folder instead of being deleted right away. This can
// be useful during rolling upgrades, for example.
// The set is backed by a concurrent HashMap.
private Set<String> trashEnabledBpids;
/** /**
* Datanode UUID that this storage is currently attached to. This * Datanode UUID that this storage is currently attached to. This
* is the same as the legacy StorageID for datanodes that were * is the same as the legacy StorageID for datanodes that were
@ -84,14 +84,16 @@ public class DataStorage extends Storage {
DataStorage() { DataStorage() {
super(NodeType.DATA_NODE); super(NodeType.DATA_NODE);
trashEnabledBpids = Collections.newSetFromMap(
new ConcurrentHashMap<String, Boolean>());
} }
public StorageInfo getBPStorage(String bpid) { public BlockPoolSliceStorage getBPStorage(String bpid) {
return bpStorageMap.get(bpid); return bpStorageMap.get(bpid);
} }
public DataStorage(StorageInfo storageInfo) { public DataStorage(StorageInfo storageInfo) {
super(NodeType.DATA_NODE, storageInfo); super(storageInfo);
} }
public synchronized String getDatanodeUuid() { public synchronized String getDatanodeUuid() {
@ -108,6 +110,43 @@ public synchronized void createStorageID(StorageDirectory sd) {
sd.setStorageUuid(DatanodeStorage.generateUuid()); sd.setStorageUuid(DatanodeStorage.generateUuid());
} }
} }
/**
* Enable trash for the specified block pool storage.
*/
public void enableTrash(String bpid) {
if (trashEnabledBpids.add(bpid)) {
LOG.info("Enabled trash for bpid " + bpid);
}
}
public void restoreTrash(String bpid) {
if (trashEnabledBpids.contains(bpid)) {
getBPStorage(bpid).restoreTrash();
trashEnabledBpids.remove(bpid);
LOG.info("Restored trash for bpid " + bpid);
}
}
public boolean trashEnabled(String bpid) {
return trashEnabledBpids.contains(bpid);
}
/**
* If rolling upgrades are in progress then do not delete block files
* immediately. Instead we move the block files to an intermediate
* 'trash' directory. If there is a subsequent rollback, then the block
* files will be restored from trash.
*
* @return trash directory if rolling upgrade is in progress, null
* otherwise.
*/
public String getTrashDirectoryForBlockFile(String bpid, File blockFile) {
if (trashEnabledBpids.contains(bpid)) {
return ((BlockPoolSliceStorage) getBPStorage(bpid)).getTrashDirectory(blockFile);
}
return null;
}
/** /**
* Analyze storage directories. * Analyze storage directories.
@ -131,10 +170,8 @@ synchronized void recoverTransitionRead(DataNode datanode,
// DN storage has been initialized, no need to do anything // DN storage has been initialized, no need to do anything
return; return;
} }
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() : LOG.info("Data-node version: " + HdfsConstants.DATANODE_LAYOUT_VERSION
"Data-node version " + HdfsConstants.LAYOUT_VERSION + + " and name-node layout version: " + nsInfo.getLayoutVersion());
" and name-node layout version " + nsInfo.getLayoutVersion() +
" must be the same.";
// 1. For each data directory calculate its state and // 1. For each data directory calculate its state and
// check whether all is consistent before transitioning. // check whether all is consistent before transitioning.
@ -186,15 +223,13 @@ synchronized void recoverTransitionRead(DataNode datanode,
// while others could be uptodate for the regular startup. // while others could be uptodate for the regular startup.
for(int idx = 0; idx < getNumStorageDirs(); idx++) { for(int idx = 0; idx < getNumStorageDirs(); idx++) {
doTransition(datanode, getStorageDir(idx), nsInfo, startOpt); doTransition(datanode, getStorageDir(idx), nsInfo, startOpt);
assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
"Data-node and name-node layout versions must be the same.";
createStorageID(getStorageDir(idx)); createStorageID(getStorageDir(idx));
} }
// 3. Update all storages. Some of them might have just been formatted. // 3. Update all storages. Some of them might have just been formatted.
this.writeAll(); this.writeAll();
// 4. mark DN storage is initilized // 4. mark DN storage is initialized
this.initialized = true; this.initialized = true;
} }
@ -261,7 +296,7 @@ static void makeBlockPoolDataDir(Collection<File> dataDirs,
void format(StorageDirectory sd, NamespaceInfo nsInfo, void format(StorageDirectory sd, NamespaceInfo nsInfo,
String datanodeUuid) throws IOException { String datanodeUuid) throws IOException {
sd.clearDirectory(); // create directory sd.clearDirectory(); // create directory
this.layoutVersion = HdfsConstants.LAYOUT_VERSION; this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
this.clusterID = nsInfo.getClusterID(); this.clusterID = nsInfo.getClusterID();
this.namespaceID = nsInfo.getNamespaceID(); this.namespaceID = nsInfo.getNamespaceID();
this.cTime = 0; this.cTime = 0;
@ -297,7 +332,8 @@ protected void setPropertiesFromFields(Properties props,
} }
// Set NamespaceID in version before federation // Set NamespaceID in version before federation
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) { if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
props.setProperty("namespaceID", String.valueOf(namespaceID)); props.setProperty("namespaceID", String.valueOf(namespaceID));
} }
} }
@ -321,11 +357,12 @@ private void setFieldsFromProperties(Properties props, StorageDirectory sd,
setLayoutVersion(props, sd); setLayoutVersion(props, sd);
} }
setcTime(props, sd); setcTime(props, sd);
setStorageType(props, sd); checkStorageType(props, sd);
setClusterId(props, layoutVersion, sd); setClusterId(props, layoutVersion, sd);
// Read NamespaceID in version before federation // Read NamespaceID in version before federation
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) { if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
setNamespaceID(props, sd); setNamespaceID(props, sd);
} }
@ -414,11 +451,12 @@ private void doTransition( DataNode datanode,
} }
readProperties(sd); readProperties(sd);
checkVersionUpgradable(this.layoutVersion); checkVersionUpgradable(this.layoutVersion);
assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION : assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION :
"Future version is not allowed"; "Future version is not allowed";
boolean federationSupported = boolean federationSupported =
LayoutVersion.supports(Feature.FEDERATION, layoutVersion); DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion);
// For pre-federation version - validate the namespaceID // For pre-federation version - validate the namespaceID
if (!federationSupported && if (!federationSupported &&
getNamespaceID() != nsInfo.getNamespaceID()) { getNamespaceID() != nsInfo.getNamespaceID()) {
@ -440,24 +478,22 @@ private void doTransition( DataNode datanode,
// meaningful at BlockPoolSliceStorage level. // meaningful at BlockPoolSliceStorage level.
// regular start up. // regular start up.
if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION) if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION)
return; // regular startup return; // regular startup
// do upgrade // do upgrade
if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION) { if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
doUpgrade(sd, nsInfo); // upgrade doUpgrade(sd, nsInfo); // upgrade
return; return;
} }
// layoutVersion < LAYOUT_VERSION. I.e. stored layout version is newer // layoutVersion < DATANODE_LAYOUT_VERSION. I.e. stored layout version is newer
// than the version supported by datanode. This should have been caught // than the version supported by datanode. This should have been caught
// in readProperties(), even if rollback was not carried out or somehow // in readProperties(), even if rollback was not carried out or somehow
// failed. // failed.
throw new IOException("BUG: The stored LV = " + this.getLayoutVersion() throw new IOException("BUG: The stored LV = " + this.getLayoutVersion()
+ " is newer than the supported LV = " + " is newer than the supported LV = "
+ HdfsConstants.LAYOUT_VERSION + HdfsConstants.DATANODE_LAYOUT_VERSION);
+ " or name node LV = "
+ nsInfo.getLayoutVersion());
} }
/** /**
@ -485,12 +521,14 @@ private void doTransition( DataNode datanode,
void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException { void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
// If the existing on-disk layout version supportes federation, simply // If the existing on-disk layout version supportes federation, simply
// update its layout version. // update its layout version.
if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) { if (DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
// The VERSION file is already read in. Override the layoutVersion // The VERSION file is already read in. Override the layoutVersion
// field and overwrite the file. // field and overwrite the file.
LOG.info("Updating layout version from " + layoutVersion + " to " LOG.info("Updating layout version from " + layoutVersion + " to "
+ nsInfo.getLayoutVersion() + " for storage " + sd.getRoot()); + HdfsConstants.DATANODE_LAYOUT_VERSION + " for storage "
layoutVersion = nsInfo.getLayoutVersion(); + sd.getRoot());
layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
writeProperties(sd); writeProperties(sd);
return; return;
} }
@ -498,7 +536,7 @@ void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
LOG.info("Upgrading storage directory " + sd.getRoot() LOG.info("Upgrading storage directory " + sd.getRoot()
+ ".\n old LV = " + this.getLayoutVersion() + ".\n old LV = " + this.getLayoutVersion()
+ "; old CTime = " + this.getCTime() + "; old CTime = " + this.getCTime()
+ ".\n new LV = " + nsInfo.getLayoutVersion() + ".\n new LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+ "; new CTime = " + nsInfo.getCTime()); + "; new CTime = " + nsInfo.getCTime());
File curDir = sd.getCurrentDir(); File curDir = sd.getCurrentDir();
@ -528,7 +566,7 @@ void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
linkAllBlocks(tmpDir, bbwDir, new File(curBpDir, STORAGE_DIR_CURRENT)); linkAllBlocks(tmpDir, bbwDir, new File(curBpDir, STORAGE_DIR_CURRENT));
// 4. Write version file under <SD>/current // 4. Write version file under <SD>/current
layoutVersion = HdfsConstants.LAYOUT_VERSION; layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
clusterID = nsInfo.getClusterID(); clusterID = nsInfo.getClusterID();
writeProperties(sd); writeProperties(sd);
@ -548,7 +586,8 @@ void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
* @throws IOException if the directory is not empty or it can not be removed * @throws IOException if the directory is not empty or it can not be removed
*/ */
private void cleanupDetachDir(File detachDir) throws IOException { private void cleanupDetachDir(File detachDir) throws IOException {
if (!LayoutVersion.supports(Feature.APPEND_RBW_DIR, layoutVersion) && if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.APPEND_RBW_DIR, layoutVersion) &&
detachDir.exists() && detachDir.isDirectory() ) { detachDir.exists() && detachDir.isDirectory() ) {
if (FileUtil.list(detachDir).length != 0 ) { if (FileUtil.list(detachDir).length != 0 ) {
@ -584,19 +623,13 @@ void doRollback( StorageDirectory sd,
File prevDir = sd.getPreviousDir(); File prevDir = sd.getPreviousDir();
// This is a regular startup or a post-federation rollback // This is a regular startup or a post-federation rollback
if (!prevDir.exists()) { if (!prevDir.exists()) {
// The current datanode version supports federation and the layout if (DataNodeLayoutVersion.supports(LayoutVersion.Feature.FEDERATION,
// version from namenode matches what the datanode supports. An invalid HdfsConstants.DATANODE_LAYOUT_VERSION)) {
// rollback may happen if namenode didn't rollback and datanode is readProperties(sd, HdfsConstants.DATANODE_LAYOUT_VERSION);
// running a wrong version. But this will be detected in block pool
// level and the invalid VERSION content will be overwritten when
// the error is corrected and rollback is retried.
if (LayoutVersion.supports(Feature.FEDERATION,
HdfsConstants.LAYOUT_VERSION) &&
HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion()) {
readProperties(sd, nsInfo.getLayoutVersion());
writeProperties(sd); writeProperties(sd);
LOG.info("Layout version rolled back to " + LOG.info("Layout version rolled back to "
nsInfo.getLayoutVersion() + " for storage " + sd.getRoot()); + HdfsConstants.DATANODE_LAYOUT_VERSION + " for storage "
+ sd.getRoot());
} }
return; return;
} }
@ -605,16 +638,17 @@ void doRollback( StorageDirectory sd,
// We allow rollback to a state, which is either consistent with // We allow rollback to a state, which is either consistent with
// the namespace state or can be further upgraded to it. // the namespace state or can be further upgraded to it.
if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION
&& prevInfo.getCTime() <= nsInfo.getCTime())) // cannot rollback && prevInfo.getCTime() <= nsInfo.getCTime())) // cannot rollback
throw new InconsistentFSStateException(sd.getRoot(), throw new InconsistentFSStateException(sd.getRoot(),
"Cannot rollback to a newer state.\nDatanode previous state: LV = " "Cannot rollback to a newer state.\nDatanode previous state: LV = "
+ prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime() + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
+ " is newer than the namespace state: LV = " + " is newer than the namespace state: LV = "
+ nsInfo.getLayoutVersion() + " CTime = " + nsInfo.getCTime()); + HdfsConstants.DATANODE_LAYOUT_VERSION + " CTime = "
+ nsInfo.getCTime());
LOG.info("Rolling back storage directory " + sd.getRoot() LOG.info("Rolling back storage directory " + sd.getRoot()
+ ".\n target LV = " + nsInfo.getLayoutVersion() + ".\n target LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+ "; target CTime = " + nsInfo.getCTime()); + "; target CTime = " + nsInfo.getCTime());
File tmpDir = sd.getRemovedTmp(); File tmpDir = sd.getRemovedTmp();
assert !tmpDir.exists() : "removed.tmp directory must not exist."; assert !tmpDir.exists() : "removed.tmp directory must not exist.";
// rename current to tmp // rename current to tmp
@ -677,9 +711,11 @@ public void run() {
/* /*
* Finalize the upgrade for a block pool * Finalize the upgrade for a block pool
* This also empties trash created during rolling upgrade and disables
* trash functionality.
*/ */
void finalizeUpgrade(String bpID) throws IOException { void finalizeUpgrade(String bpID) throws IOException {
// To handle finalizing a snapshot taken at datanode level while // To handle finalizing a snapshot taken at datanode level while
// upgrading to federation, if datanode level snapshot previous exists, // upgrading to federation, if datanode level snapshot previous exists,
// then finalize it. Else finalize the corresponding BP. // then finalize it. Else finalize the corresponding BP.
for (StorageDirectory sd : storageDirs) { for (StorageDirectory sd : storageDirs) {
@ -710,7 +746,8 @@ private void linkAllBlocks(File fromDir, File fromBbwDir, File toDir)
HardLink hardLink = new HardLink(); HardLink hardLink = new HardLink();
// do the link // do the link
int diskLayoutVersion = this.getLayoutVersion(); int diskLayoutVersion = this.getLayoutVersion();
if (LayoutVersion.supports(Feature.APPEND_RBW_DIR, diskLayoutVersion)) { if (DataNodeLayoutVersion.supports(
LayoutVersion.Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
// hardlink finalized blocks in tmpDir/finalized // hardlink finalized blocks in tmpDir/finalized
linkBlocks(new File(fromDir, STORAGE_DIR_FINALIZED), linkBlocks(new File(fromDir, STORAGE_DIR_FINALIZED),
new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink); new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);

View File

@ -166,8 +166,8 @@ public void run() {
int opsProcessed = 0; int opsProcessed = 0;
Op op = null; Op op = null;
dataXceiverServer.addPeer(peer);
try { try {
dataXceiverServer.addPeer(peer, Thread.currentThread());
peer.setWriteTimeout(datanode.getDnConf().socketWriteTimeout); peer.setWriteTimeout(datanode.getDnConf().socketWriteTimeout);
InputStream input = socketIn; InputStream input = socketIn;
if ((!peer.hasSecureChannel()) && dnConf.encryptDataTransfer) { if ((!peer.hasSecureChannel()) && dnConf.encryptDataTransfer) {

View File

@ -20,8 +20,7 @@
import java.io.IOException; import java.io.IOException;
import java.net.SocketTimeoutException; import java.net.SocketTimeoutException;
import java.nio.channels.AsynchronousCloseException; import java.nio.channels.AsynchronousCloseException;
import java.util.HashSet; import java.util.HashMap;
import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -45,7 +44,8 @@ class DataXceiverServer implements Runnable {
private final PeerServer peerServer; private final PeerServer peerServer;
private final DataNode datanode; private final DataNode datanode;
private final Set<Peer> peers = new HashSet<Peer>(); private final HashMap<Peer, Thread> peers = new HashMap<Peer, Thread>();
private boolean closed = false;
/** /**
* Maximal number of concurrent xceivers per node. * Maximal number of concurrent xceivers per node.
@ -127,7 +127,7 @@ synchronized void release() {
@Override @Override
public void run() { public void run() {
Peer peer = null; Peer peer = null;
while (datanode.shouldRun) { while (datanode.shouldRun && !datanode.shutdownForUpgrade) {
try { try {
peer = peerServer.accept(); peer = peerServer.accept();
@ -147,7 +147,7 @@ public void run() {
} catch (AsynchronousCloseException ace) { } catch (AsynchronousCloseException ace) {
// another thread closed our listener socket - that's expected during shutdown, // another thread closed our listener socket - that's expected during shutdown,
// but not in other circumstances // but not in other circumstances
if (datanode.shouldRun) { if (datanode.shouldRun && !datanode.shutdownForUpgrade) {
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ace); LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ace);
} }
} catch (IOException ie) { } catch (IOException ie) {
@ -170,31 +170,53 @@ public void run() {
datanode.shouldRun = false; datanode.shouldRun = false;
} }
} }
synchronized (this) {
for (Peer p : peers) { // Close the server to stop reception of more requests.
IOUtils.cleanup(LOG, p);
}
}
try { try {
peerServer.close(); peerServer.close();
closed = true;
} catch (IOException ie) { } catch (IOException ie) {
LOG.warn(datanode.getDisplayName() LOG.warn(datanode.getDisplayName()
+ " :DataXceiverServer: close exception", ie); + " :DataXceiverServer: close exception", ie);
} }
// if in restart prep stage, notify peers before closing them.
if (datanode.shutdownForUpgrade) {
restartNotifyPeers();
// Each thread needs some time to process it. If a thread needs
// to send an OOB message to the client, but blocked on network for
// long time, we need to force its termination.
LOG.info("Shutting down DataXceiverServer before restart");
// Allow roughly up to 2 seconds.
for (int i = 0; getNumPeers() > 0 && i < 10; i++) {
try {
Thread.sleep(200);
} catch (InterruptedException e) {
// ignore
}
}
}
// Close all peers.
closeAllPeers();
} }
void kill() { void kill() {
assert datanode.shouldRun == false : assert (datanode.shouldRun == false || datanode.shutdownForUpgrade) :
"shoudRun should be set to false before killing"; "shoudRun should be set to false or restarting should be true"
+ " before killing";
try { try {
this.peerServer.close(); this.peerServer.close();
this.closed = true;
} catch (IOException ie) { } catch (IOException ie) {
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer.kill(): ", ie); LOG.warn(datanode.getDisplayName() + ":DataXceiverServer.kill(): ", ie);
} }
} }
synchronized void addPeer(Peer peer) { synchronized void addPeer(Peer peer, Thread t) throws IOException {
peers.add(peer); if (closed) {
throw new IOException("Server closed.");
}
peers.put(peer, t);
} }
synchronized void closePeer(Peer peer) { synchronized void closePeer(Peer peer) {
@ -202,6 +224,31 @@ synchronized void closePeer(Peer peer) {
IOUtils.cleanup(null, peer); IOUtils.cleanup(null, peer);
} }
// Notify all peers of the shutdown and restart.
// datanode.shouldRun should still be true and datanode.restarting should
// be set true before calling this method.
synchronized void restartNotifyPeers() {
assert (datanode.shouldRun == true && datanode.shutdownForUpgrade);
for (Peer p : peers.keySet()) {
// interrupt each and every DataXceiver thread.
peers.get(p).interrupt();
}
}
// Close all peers and clear the map.
synchronized void closeAllPeers() {
LOG.info("Closing all peers.");
for (Peer p : peers.keySet()) {
IOUtils.cleanup(LOG, p);
}
peers.clear();
}
// Return the number of peers.
synchronized int getNumPeers() {
return peers.size();
}
synchronized void releasePeer(Peer peer) { synchronized void releasePeer(Peer peer) {
peers.remove(peer); peers.remove(peer);
} }

View File

@ -412,5 +412,22 @@ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
*/ */
public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid, public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
long[] blockIds) throws IOException; long[] blockIds) throws IOException;
/**
* Enable 'trash' for the given dataset. When trash is enabled, files are
* moved to a separate trash directory instead of being deleted immediately.
* This can be useful for example during rolling upgrades.
*/
public void enableTrash(String bpid);
/**
* Restore trash
*/
public void restoreTrash(String bpid);
/**
* @return true when trash is enabled
*/
public boolean trashEnabled(String bpid);
} }

View File

@ -21,9 +21,12 @@
import java.io.DataInputStream; 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.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.util.Scanner;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DU; import org.apache.hadoop.fs.DU;
@ -36,11 +39,14 @@
import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil; import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica; import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered; import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DiskChecker; import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.Time;
/** /**
* A block pool slice represents a portion of a block pool stored on a volume. * A block pool slice represents a portion of a block pool stored on a volume.
@ -56,6 +62,9 @@ class BlockPoolSlice {
private final LDir finalizedDir; // directory store Finalized replica private final LDir finalizedDir; // directory store Finalized replica
private final File rbwDir; // directory store RBW replica private final File rbwDir; // directory store RBW replica
private final File tmpDir; // directory store Temporary replica private final File tmpDir; // directory store Temporary replica
private static String DU_CACHE_FILE = "dfsUsed";
private volatile boolean dfsUsedSaved = false;
private static final int SHUTDOWN_HOOK_PRIORITY = 30;
// TODO:FEDERATION scalability issue - a thread per DU is needed // TODO:FEDERATION scalability issue - a thread per DU is needed
private final DU dfsUsage; private final DU dfsUsage;
@ -106,8 +115,21 @@ class BlockPoolSlice {
throw new IOException("Mkdirs failed to create " + tmpDir.toString()); throw new IOException("Mkdirs failed to create " + tmpDir.toString());
} }
} }
this.dfsUsage = new DU(bpDir, conf); // Use cached value initially if available. Or the following call will
// block until the initial du command completes.
this.dfsUsage = new DU(bpDir, conf, loadDfsUsed());
this.dfsUsage.start(); this.dfsUsage.start();
// Make the dfs usage to be saved during shutdown.
ShutdownHookManager.get().addShutdownHook(
new Runnable() {
@Override
public void run() {
if (!dfsUsedSaved) {
saveDfsUsed();
}
}
}, SHUTDOWN_HOOK_PRIORITY);
} }
File getDirectory() { File getDirectory() {
@ -131,6 +153,79 @@ long getDfsUsed() throws IOException {
return dfsUsage.getUsed(); return dfsUsage.getUsed();
} }
/**
* Read in the cached DU value and return it if it is less than 600 seconds
* old (DU update interval). Slight imprecision of dfsUsed is not critical
* and skipping DU can significantly shorten the startup time.
* If the cached value is not available or too old, -1 is returned.
*/
long loadDfsUsed() {
long cachedDfsUsed;
long mtime;
Scanner sc;
try {
sc = new Scanner(new File(currentDir, DU_CACHE_FILE));
} catch (FileNotFoundException fnfe) {
return -1;
}
try {
// Get the recorded dfsUsed from the file.
if (sc.hasNextLong()) {
cachedDfsUsed = sc.nextLong();
} else {
return -1;
}
// Get the recorded mtime from the file.
if (sc.hasNextLong()) {
mtime = sc.nextLong();
} else {
return -1;
}
// Return the cached value if mtime is okay.
if (mtime > 0 && (Time.now() - mtime < 600000L)) {
FsDatasetImpl.LOG.info("Cached dfsUsed found for " + currentDir + ": " +
cachedDfsUsed);
return cachedDfsUsed;
}
return -1;
} finally {
sc.close();
}
}
/**
* Write the current dfsUsed to the cache file.
*/
void saveDfsUsed() {
File outFile = new File(currentDir, DU_CACHE_FILE);
if (outFile.exists() && !outFile.delete()) {
FsDatasetImpl.LOG.warn("Failed to delete old dfsUsed file in " +
outFile.getParent());
}
FileWriter out = null;
try {
long used = getDfsUsed();
if (used > 0) {
out = new FileWriter(outFile);
// mtime is written last, so that truncated writes won't be valid.
out.write(Long.toString(used) + " " + Long.toString(Time.now()));
out.flush();
out.close();
out = null;
}
} catch (IOException ioe) {
// If write failed, the volume might be bad. Since the cache file is
// not critical, log the error and continue.
FsDatasetImpl.LOG.warn("Failed to write dfsUsed to " + outFile, ioe);
} finally {
IOUtils.cleanup(null, out);
}
}
/** /**
* Temporary files. They get moved to the finalized block directory when * Temporary files. They get moved to the finalized block directory when
* the block is finalized. * the block is finalized.
@ -191,9 +286,39 @@ void addToReplicasMap(ReplicaMap volumeMap, File dir, boolean isFinalized
newReplica = new FinalizedReplica(blockId, newReplica = new FinalizedReplica(blockId,
blockFile.length(), genStamp, volume, blockFile.getParentFile()); blockFile.length(), genStamp, volume, blockFile.getParentFile());
} else { } else {
newReplica = new ReplicaWaitingToBeRecovered(blockId,
validateIntegrityAndSetLength(blockFile, genStamp), boolean loadRwr = true;
genStamp, volume, blockFile.getParentFile()); File restartMeta = new File(blockFile.getParent() +
File.pathSeparator + "." + blockFile.getName() + ".restart");
Scanner sc = null;
try {
sc = new Scanner(restartMeta);
// The restart meta file exists
if (sc.hasNextLong() && (sc.nextLong() > Time.now())) {
// It didn't expire. Load the replica as a RBW.
newReplica = new ReplicaBeingWritten(blockId,
validateIntegrityAndSetLength(blockFile, genStamp),
genStamp, volume, blockFile.getParentFile(), null);
loadRwr = false;
}
sc.close();
if (restartMeta.delete()) {
FsDatasetImpl.LOG.warn("Failed to delete restart meta file: " +
restartMeta.getPath());
}
} catch (FileNotFoundException fnfe) {
// nothing to do here
} finally {
if (sc != null) {
sc.close();
}
}
// Restart meta doesn't exist or expired.
if (loadRwr) {
newReplica = new ReplicaWaitingToBeRecovered(blockId,
validateIntegrityAndSetLength(blockFile, genStamp),
genStamp, volume, blockFile.getParentFile());
}
} }
ReplicaInfo oldReplica = volumeMap.add(bpid, newReplica); ReplicaInfo oldReplica = volumeMap.add(bpid, newReplica);
@ -296,6 +421,8 @@ public String toString() {
} }
void shutdown() { void shutdown() {
saveDfsUsed();
dfsUsedSaved = true;
dfsUsage.shutdown(); dfsUsage.shutdown();
} }
} }

View File

@ -153,29 +153,34 @@ synchronized void shutdown() {
* dfsUsed statistics accordingly. * dfsUsed statistics accordingly.
*/ */
void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile, void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
ExtendedBlock block) { ExtendedBlock block, String trashDirectory) {
LOG.info("Scheduling " + block.getLocalBlock() LOG.info("Scheduling " + block.getLocalBlock()
+ " file " + blockFile + " for deletion"); + " file " + blockFile + " for deletion");
ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask( ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
volume, blockFile, metaFile, block); volume, blockFile, metaFile, block, trashDirectory);
execute(volume.getCurrentDir(), deletionTask); execute(volume.getCurrentDir(), deletionTask);
} }
/** A task for deleting a block file and its associated meta file, as well /** A task for deleting a block file and its associated meta file, as well
* as decrement the dfs usage of the volume. * as decrement the dfs usage of the volume.
* Optionally accepts a trash directory. If one is specified then the files
* are moved to trash instead of being deleted. If none is specified then the
* files are deleted immediately.
*/ */
class ReplicaFileDeleteTask implements Runnable { class ReplicaFileDeleteTask implements Runnable {
final FsVolumeImpl volume; final FsVolumeImpl volume;
final File blockFile; final File blockFile;
final File metaFile; final File metaFile;
final ExtendedBlock block; final ExtendedBlock block;
final String trashDirectory;
ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile, ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile,
File metaFile, ExtendedBlock block) { File metaFile, ExtendedBlock block, String trashDirectory) {
this.volume = volume; this.volume = volume;
this.blockFile = blockFile; this.blockFile = blockFile;
this.metaFile = metaFile; this.metaFile = metaFile;
this.block = block; this.block = block;
this.trashDirectory = trashDirectory;
} }
@Override @Override
@ -186,12 +191,39 @@ public String toString() {
+ " and meta file " + metaFile + " from volume " + volume; + " and meta file " + metaFile + " from volume " + volume;
} }
private boolean deleteFiles() {
return blockFile.delete() && (metaFile.delete() || !metaFile.exists());
}
private boolean moveFiles() {
File trashDirFile = new File(trashDirectory);
if (!trashDirFile.exists() && !trashDirFile.mkdirs()) {
LOG.error("Failed to create trash directory " + trashDirectory);
return false;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Moving files " + blockFile.getName() + " and " +
metaFile.getName() + " to trash.");
}
File newBlockFile = new File(trashDirectory, blockFile.getName());
File newMetaFile = new File(trashDirectory, metaFile.getName());
return (blockFile.renameTo(newBlockFile) &&
metaFile.renameTo(newMetaFile));
}
@Override @Override
public void run() { public void run() {
long dfsBytes = blockFile.length() + metaFile.length(); long dfsBytes = blockFile.length() + metaFile.length();
if (!blockFile.delete() || (!metaFile.delete() && metaFile.exists())) { boolean result;
LOG.warn("Unexpected error trying to delete block "
+ block.getBlockPoolId() + " " + block.getLocalBlock() result = (trashDirectory == null) ? deleteFiles() : moveFiles();
if (!result) {
LOG.warn("Unexpected error trying to "
+ (trashDirectory == null ? "delete" : "move")
+ " block " + block.getBlockPoolId() + " " + block.getLocalBlock()
+ " at file " + blockFile + ". Ignored."); + " at file " + blockFile + ". Ignored.");
} else { } else {
if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){ if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){

View File

@ -17,28 +17,6 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import javax.management.NotCompliantMBeanException;
import javax.management.ObjectName;
import javax.management.StandardMBean;
import com.google.common.base.Preconditions;
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.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -46,37 +24,12 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.hdfs.server.datanode.*;
import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner; import org.apache.hadoop.hdfs.server.datanode.fsdataset.*;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
import org.apache.hadoop.hdfs.server.datanode.Replica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean; import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@ -90,6 +43,15 @@
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import javax.management.NotCompliantMBeanException;
import javax.management.ObjectName;
import javax.management.StandardMBean;
import java.io.*;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.*;
import java.util.concurrent.Executor;
/************************************************** /**************************************************
* FSDataset manages a set of data blocks. Each block * FSDataset manages a set of data blocks. Each block
* has a unique name and an extent on disk. * has a unique name and an extent on disk.
@ -193,6 +155,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
} }
final DataNode datanode; final DataNode datanode;
final DataStorage dataStorage;
final FsVolumeList volumes; final FsVolumeList volumes;
final FsDatasetAsyncDiskService asyncDiskService; final FsDatasetAsyncDiskService asyncDiskService;
final FsDatasetCache cacheManager; final FsDatasetCache cacheManager;
@ -209,6 +172,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
FsDatasetImpl(DataNode datanode, DataStorage storage, Configuration conf FsDatasetImpl(DataNode datanode, DataStorage storage, Configuration conf
) throws IOException { ) throws IOException {
this.datanode = datanode; this.datanode = datanode;
this.dataStorage = storage;
// The number of volumes required for operation is the total number // The number of volumes required for operation is the total number
// of volumes minus the number of failed volumes we can tolerate. // of volumes minus the number of failed volumes we can tolerate.
final int volFailuresTolerated = final int volFailuresTolerated =
@ -1234,7 +1198,8 @@ public void invalidate(String bpid, Block invalidBlks[]) throws IOException {
// finishes. // finishes.
asyncDiskService.deleteAsync(v, f, asyncDiskService.deleteAsync(v, f,
FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()), FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
new ExtendedBlock(bpid, invalidBlks[i])); new ExtendedBlock(bpid, invalidBlks[i]),
dataStorage.getTrashDirectoryForBlockFile(bpid, f));
} }
if (error) { if (error) {
throw new IOException("Error in deleting blocks."); throw new IOException("Error in deleting blocks.");
@ -1762,11 +1727,13 @@ public synchronized long getReplicaVisibleLength(final ExtendedBlock block)
} }
@Override @Override
public synchronized void addBlockPool(String bpid, Configuration conf) public void addBlockPool(String bpid, Configuration conf)
throws IOException { throws IOException {
LOG.info("Adding block pool " + bpid); LOG.info("Adding block pool " + bpid);
volumes.addBlockPool(bpid, conf); synchronized(this) {
volumeMap.initBlockPool(bpid); volumes.addBlockPool(bpid, conf);
volumeMap.initBlockPool(bpid);
}
volumes.getAllVolumesMap(bpid, volumeMap); volumes.getAllVolumesMap(bpid, volumeMap);
} }
@ -1895,6 +1862,21 @@ public HdfsBlocksMetadata getHdfsBlocksMetadata(String poolId,
blocksVolumeIds, blocksVolumeIndexes); blocksVolumeIds, blocksVolumeIndexes);
} }
@Override
public void enableTrash(String bpid) {
dataStorage.enableTrash(bpid);
}
@Override
public void restoreTrash(String bpid) {
dataStorage.restoreTrash(bpid);
}
@Override
public boolean trashEnabled(String bpid) {
return dataStorage.trashEnabled(bpid);
}
@Override @Override
public RollingLogs createRollingLogs(String bpid, String prefix public RollingLogs createRollingLogs(String bpid, String prefix
) throws IOException { ) throws IOException {

View File

@ -96,10 +96,41 @@ void initializeReplicaMaps(ReplicaMap globalReplicaMap) throws IOException {
} }
} }
void getAllVolumesMap(String bpid, ReplicaMap volumeMap) throws IOException { void getAllVolumesMap(final String bpid, final ReplicaMap volumeMap) throws IOException {
long totalStartTime = System.currentTimeMillis(); long totalStartTime = System.currentTimeMillis();
for (FsVolumeImpl v : volumes) { final List<IOException> exceptions = Collections.synchronizedList(
getVolumeMap(bpid, v, volumeMap); new ArrayList<IOException>());
List<Thread> replicaAddingThreads = new ArrayList<Thread>();
for (final FsVolumeImpl v : volumes) {
Thread t = new Thread() {
public void run() {
try {
FsDatasetImpl.LOG.info("Adding replicas to map for block pool " +
bpid + " on volume " + v + "...");
long startTime = System.currentTimeMillis();
v.getVolumeMap(bpid, volumeMap);
long timeTaken = System.currentTimeMillis() - startTime;
FsDatasetImpl.LOG.info("Time to add replicas to map for block pool"
+ " " + bpid + " on volume " + v + ": " + timeTaken + "ms");
} catch (IOException ioe) {
FsDatasetImpl.LOG.info("Caught exception while adding replicas " +
"from " + v + ". Will throw later.", ioe);
exceptions.add(ioe);
}
}
};
replicaAddingThreads.add(t);
t.start();
}
for (Thread t : replicaAddingThreads) {
try {
t.join();
} catch (InterruptedException ie) {
throw new IOException(ie);
}
}
if (!exceptions.isEmpty()) {
throw exceptions.get(0);
} }
long totalTimeTaken = System.currentTimeMillis() - totalStartTime; long totalTimeTaken = System.currentTimeMillis() - totalStartTime;
FsDatasetImpl.LOG.info("Total time to add all replicas to map: " FsDatasetImpl.LOG.info("Total time to add all replicas to map: "
@ -219,4 +250,4 @@ void shutdown() {
} }
} }
} }
} }

View File

@ -221,7 +221,7 @@ private synchronized void applyEdits(long firstTxId, int numTxns, byte[] data)
backupInputStream.setBytes(data, logVersion); backupInputStream.setBytes(data, logVersion);
long numTxnsAdvanced = logLoader.loadEditRecords( long numTxnsAdvanced = logLoader.loadEditRecords(
backupInputStream, true, lastAppliedTxId + 1, null); backupInputStream, true, lastAppliedTxId + 1, null, null);
if (numTxnsAdvanced != numTxns) { if (numTxnsAdvanced != numTxns) {
throw new IOException("Batch of txns starting at txnid " + throw new IOException("Batch of txns starting at txnid " +
firstTxId + " was supposed to contain " + numTxns + firstTxId + " was supposed to contain " + numTxns +
@ -279,7 +279,7 @@ private boolean tryConvergeJournalSpool() throws IOException {
editStreams.add(s); editStreams.add(s);
} }
} }
loadEdits(editStreams, namesystem, null); loadEdits(editStreams, namesystem);
} }
// now, need to load the in-progress file // now, need to load the in-progress file
@ -315,7 +315,7 @@ private boolean tryConvergeJournalSpool() throws IOException {
FSEditLogLoader loader = FSEditLogLoader loader =
new FSEditLogLoader(namesystem, lastAppliedTxId); new FSEditLogLoader(namesystem, lastAppliedTxId);
loader.loadFSEdits(stream, lastAppliedTxId + 1, null); loader.loadFSEdits(stream, lastAppliedTxId + 1);
lastAppliedTxId = loader.getLastAppliedTxId(); lastAppliedTxId = loader.getLastAppliedTxId();
assert lastAppliedTxId == getEditLog().getLastWrittenTxId(); assert lastAppliedTxId == getEditLog().getLastWrittenTxId();
} finally { } finally {

View File

@ -97,4 +97,9 @@ public boolean matchesRegistration(NamenodeRegistration bnReg) {
public String toString() { public String toString() {
return "BackupJournalManager"; return "BackupJournalManager";
} }
@Override
public void discardSegments(long startTxId) throws IOException {
throw new UnsupportedOperationException();
}
} }

View File

@ -30,12 +30,14 @@
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.NameNodeProxies; import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService; import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB; import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.ha.HAState; import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
import org.apache.hadoop.hdfs.server.protocol.FenceResponse; import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
@ -44,8 +46,8 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
@ -357,7 +359,7 @@ private void registerWith(NamespaceInfo nsInfo) throws IOException {
} else { } else {
nsInfo.validateStorage(storage); nsInfo.validateStorage(storage);
} }
bnImage.initEditLog(); bnImage.initEditLog(StartupOption.REGULAR);
setRegistration(); setRegistration();
NamenodeRegistration nnReg = null; NamenodeRegistration nnReg = null;
while(!isStopRequested()) { while(!isStopRequested()) {
@ -402,9 +404,9 @@ private static NamespaceInfo handshake(NamenodeProtocol namenode)
LOG.fatal(errorMsg); LOG.fatal(errorMsg);
throw new IOException(errorMsg); throw new IOException(errorMsg);
} }
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() : assert HdfsConstants.NAMENODE_LAYOUT_VERSION == nsInfo.getLayoutVersion() :
"Active and backup node layout versions must be the same. Expected: " "Active and backup node layout versions must be the same. Expected: "
+ HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion(); + HdfsConstants.NAMENODE_LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
return nsInfo; return nsInfo;
} }

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.namenode.FSImage; import org.apache.hadoop.hdfs.server.namenode.FSImage;
import com.google.common.collect.ComparisonChain; import com.google.common.collect.ComparisonChain;
@ -47,6 +48,7 @@ public class CheckpointSignature extends StorageInfo
} }
CheckpointSignature(String str) { CheckpointSignature(String str) {
super(NodeType.NAME_NODE);
String[] fields = str.split(FIELD_SEPARATOR); String[] fields = str.split(FIELD_SEPARATOR);
assert fields.length == NUM_FIELDS : assert fields.length == NUM_FIELDS :
"Must be " + NUM_FIELDS + " fields in CheckpointSignature"; "Must be " + NUM_FIELDS + " fields in CheckpointSignature";

View File

@ -33,6 +33,7 @@
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand; import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@ -220,9 +221,9 @@ void doCheckpoint() throws IOException {
LOG.info("Unable to roll forward using only logs. Downloading " + LOG.info("Unable to roll forward using only logs. Downloading " +
"image with txid " + sig.mostRecentCheckpointTxId); "image with txid " + sig.mostRecentCheckpointTxId);
MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage( MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId, backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId, bnStorage,
bnStorage, true); true);
bnImage.saveDigestAndRenameCheckpointImage( bnImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE,
sig.mostRecentCheckpointTxId, downloadedHash); sig.mostRecentCheckpointTxId, downloadedHash);
lastApplied = sig.mostRecentCheckpointTxId; lastApplied = sig.mostRecentCheckpointTxId;
needReloadImage = true; needReloadImage = true;
@ -240,7 +241,8 @@ void doCheckpoint() throws IOException {
if(needReloadImage) { if(needReloadImage) {
LOG.info("Loading image with txid " + sig.mostRecentCheckpointTxId); LOG.info("Loading image with txid " + sig.mostRecentCheckpointTxId);
File file = bnStorage.findImageFile(sig.mostRecentCheckpointTxId); File file = bnStorage.findImageFile(NameNodeFile.IMAGE,
sig.mostRecentCheckpointTxId);
bnImage.reloadFromImageFile(file, backupNode.getNamesystem()); bnImage.reloadFromImageFile(file, backupNode.getNamesystem());
} }
rollForwardByApplyingLogs(manifest, bnImage, backupNode.getNamesystem()); rollForwardByApplyingLogs(manifest, bnImage, backupNode.getNamesystem());
@ -263,7 +265,7 @@ void doCheckpoint() throws IOException {
if(cpCmd.needToReturnImage()) { if(cpCmd.needToReturnImage()) {
TransferFsImage.uploadImageFromStorage( TransferFsImage.uploadImageFromStorage(
backupNode.nnHttpAddress, getImageListenAddress(), backupNode.nnHttpAddress, getImageListenAddress(),
bnStorage, txid); bnStorage, NameNodeFile.IMAGE, txid);
} }
getRemoteNamenodeProxy().endCheckpoint(backupNode.getRegistration(), sig); getRemoteNamenodeProxy().endCheckpoint(backupNode.getRegistration(), sig);
@ -307,6 +309,6 @@ static void rollForwardByApplyingLogs(
} }
LOG.info("Checkpointer about to load edits from " + LOG.info("Checkpointer about to load edits from " +
editsStreams.size() + " stream(s)."); editsStreams.size() + " stream(s).");
dstImage.loadEdits(editsStreams, dstNamesystem, null); dstImage.loadEdits(editsStreams, dstNamesystem);
} }
} }

View File

@ -36,7 +36,6 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutFlags; import org.apache.hadoop.hdfs.protocol.LayoutFlags;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException; import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
import org.apache.hadoop.hdfs.web.URLConnectionFactory; import org.apache.hadoop.hdfs.web.URLConnectionFactory;
@ -150,7 +149,8 @@ private void init() throws LogHeaderCorruptException, IOException {
} catch (EOFException eofe) { } catch (EOFException eofe) {
throw new LogHeaderCorruptException("No header found in log"); throw new LogHeaderCorruptException("No header found in log");
} }
if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_LAYOUT_FLAGS, logVersion)) {
try { try {
LayoutFlags.read(dataIn); LayoutFlags.read(dataIn);
} catch (EOFException eofe) { } catch (EOFException eofe) {
@ -329,12 +329,12 @@ static int readLogVersion(DataInputStream in)
throw new LogHeaderCorruptException( throw new LogHeaderCorruptException(
"Reached EOF when reading log header"); "Reached EOF when reading log header");
} }
if (logVersion < HdfsConstants.LAYOUT_VERSION || // future version if (logVersion < HdfsConstants.NAMENODE_LAYOUT_VERSION || // future version
logVersion > Storage.LAST_UPGRADABLE_LAYOUT_VERSION) { // unsupported logVersion > Storage.LAST_UPGRADABLE_LAYOUT_VERSION) { // unsupported
throw new LogHeaderCorruptException( throw new LogHeaderCorruptException(
"Unexpected version of the file system log file: " "Unexpected version of the file system log file: "
+ logVersion + ". Current version = " + logVersion + ". Current version = "
+ HdfsConstants.LAYOUT_VERSION + "."); + HdfsConstants.NAMENODE_LAYOUT_VERSION + ".");
} }
return logVersion; return logVersion;
} }

View File

@ -132,7 +132,7 @@ public void create() throws IOException {
*/ */
@VisibleForTesting @VisibleForTesting
public static void writeHeader(DataOutputStream out) throws IOException { public static void writeHeader(DataOutputStream out) throws IOException {
out.writeInt(HdfsConstants.LAYOUT_VERSION); out.writeInt(HdfsConstants.NAMENODE_LAYOUT_VERSION);
LayoutFlags.write(out); LayoutFlags.write(out);
} }

View File

@ -20,36 +20,36 @@
import static org.apache.hadoop.util.ExitUtil.terminate; import static org.apache.hadoop.util.ExitUtil.terminate;
import static org.apache.hadoop.util.Time.now; import static org.apache.hadoop.util.Time.now;
import java.net.URI;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Constructor;
import java.net.URI;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.lang.reflect.Constructor;
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.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable; import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
@ -66,8 +66,8 @@
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
@ -83,6 +83,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RollingUpgradeOp;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream; import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@ -91,7 +92,6 @@
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.conf.Configuration;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -1036,6 +1036,18 @@ void logSetAcl(String src, List<AclEntry> entries) {
logEdit(op); logEdit(op);
} }
void logStartRollingUpgrade(long startTime) {
RollingUpgradeOp op = RollingUpgradeOp.getStartInstance(cache.get());
op.setTime(startTime);
logEdit(op);
}
void logFinalizeRollingUpgrade(long finalizeTime) {
RollingUpgradeOp op = RollingUpgradeOp.getFinalizeInstance(cache.get());
op.setTime(finalizeTime);
logEdit(op);
}
/** /**
* Get all the journals this edit log is currently operating on. * Get all the journals this edit log is currently operating on.
*/ */
@ -1318,6 +1330,13 @@ synchronized void recoverUnclosedStreams() {
} }
} }
public synchronized void discardSegments(long markerTxid)
throws IOException {
for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
jas.getManager().discardSegments(markerTxid);
}
}
@Override @Override
public void selectInputStreams(Collection<EditLogInputStream> streams, public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk) throws IOException { long fromTxId, boolean inProgressOk) throws IOException {

View File

@ -37,10 +37,11 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
@ -68,6 +69,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetAclOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetAclOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RollingUpgradeOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
@ -81,6 +83,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease; import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase; import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress; import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
@ -96,22 +99,30 @@
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class FSEditLogLoader { public class FSEditLogLoader {
static final Log LOG = LogFactory.getLog(FSEditLogLoader.class.getName()); static final Log LOG = LogFactory.getLog(FSEditLogLoader.class.getName());
static long REPLAY_TRANSACTION_LOG_INTERVAL = 1000; // 1sec static final long REPLAY_TRANSACTION_LOG_INTERVAL = 1000; // 1sec
private final FSNamesystem fsNamesys; private final FSNamesystem fsNamesys;
private long lastAppliedTxId; private long lastAppliedTxId;
/** Total number of end transactions loaded. */
private int totalEdits = 0;
public FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId) { public FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId) {
this.fsNamesys = fsNamesys; this.fsNamesys = fsNamesys;
this.lastAppliedTxId = lastAppliedTxId; this.lastAppliedTxId = lastAppliedTxId;
} }
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
throws IOException {
return loadFSEdits(edits, expectedStartingTxId, null, null);
}
/** /**
* Load an edit log, and apply the changes to the in-memory structure * Load an edit log, and apply the changes to the in-memory structure
* This is where we apply edits that we've been writing to disk all * This is where we apply edits that we've been writing to disk all
* along. * along.
*/ */
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId, long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId,
MetaRecoveryContext recovery) throws IOException { StartupOption startOpt, MetaRecoveryContext recovery) throws IOException {
StartupProgress prog = NameNode.getStartupProgress(); StartupProgress prog = NameNode.getStartupProgress();
Step step = createStartupProgressStep(edits); Step step = createStartupProgressStep(edits);
prog.beginStep(Phase.LOADING_EDITS, step); prog.beginStep(Phase.LOADING_EDITS, step);
@ -119,8 +130,8 @@ long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId,
try { try {
long startTime = now(); long startTime = now();
FSImage.LOG.info("Start loading edits file " + edits.getName()); FSImage.LOG.info("Start loading edits file " + edits.getName());
long numEdits = loadEditRecords(edits, false, long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
expectedStartingTxId, recovery); startOpt, recovery);
FSImage.LOG.info("Edits file " + edits.getName() FSImage.LOG.info("Edits file " + edits.getName()
+ " of size " + edits.length() + " edits # " + numEdits + " of size " + edits.length() + " edits # " + numEdits
+ " loaded in " + (now()-startTime)/1000 + " seconds"); + " loaded in " + (now()-startTime)/1000 + " seconds");
@ -133,8 +144,8 @@ long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId,
} }
long loadEditRecords(EditLogInputStream in, boolean closeOnExit, long loadEditRecords(EditLogInputStream in, boolean closeOnExit,
long expectedStartingTxId, MetaRecoveryContext recovery) long expectedStartingTxId, StartupOption startOpt,
throws IOException { MetaRecoveryContext recovery) throws IOException {
FSDirectory fsDir = fsNamesys.dir; FSDirectory fsDir = fsNamesys.dir;
EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts = EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts =
@ -206,12 +217,23 @@ long loadEditRecords(EditLogInputStream in, boolean closeOnExit,
} }
} }
try { try {
long inodeId = applyEditLogOp(op, fsDir, in.getVersion(), lastInodeId); if (LOG.isTraceEnabled()) {
LOG.trace("op=" + op + ", startOpt=" + startOpt
+ ", numEdits=" + numEdits + ", totalEdits=" + totalEdits);
}
long inodeId = applyEditLogOp(op, fsDir, startOpt,
in.getVersion(), lastInodeId);
if (lastInodeId < inodeId) { if (lastInodeId < inodeId) {
lastInodeId = inodeId; lastInodeId = inodeId;
} }
} catch (RollingUpgradeOp.RollbackException e) {
throw e;
} catch (Throwable e) { } catch (Throwable e) {
LOG.error("Encountered exception on operation " + op, e); LOG.error("Encountered exception on operation " + op, e);
if (recovery == null) {
throw e instanceof IOException? (IOException)e: new IOException(e);
}
MetaRecoveryContext.editLogLoaderPrompt("Failed to " + MetaRecoveryContext.editLogLoaderPrompt("Failed to " +
"apply edit log operation " + op + ": error " + "apply edit log operation " + op + ": error " +
e.getMessage(), recovery, "applying edits"); e.getMessage(), recovery, "applying edits");
@ -237,6 +259,10 @@ long loadEditRecords(EditLogInputStream in, boolean closeOnExit,
} }
} }
numEdits++; numEdits++;
totalEdits++;
} catch (RollingUpgradeOp.RollbackException e) {
LOG.info("Stopped at OP_START_ROLLING_UPGRADE for rollback.");
break;
} catch (MetaRecoveryContext.RequestStopException e) { } catch (MetaRecoveryContext.RequestStopException e) {
MetaRecoveryContext.LOG.warn("Stopped reading edit log at " + MetaRecoveryContext.LOG.warn("Stopped reading edit log at " +
in.getPosition() + "/" + in.length()); in.getPosition() + "/" + in.length());
@ -268,7 +294,8 @@ private long getAndUpdateLastInodeId(long inodeIdFromOp, int logVersion,
long inodeId = inodeIdFromOp; long inodeId = inodeIdFromOp;
if (inodeId == INodeId.GRANDFATHER_INODE_ID) { if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
throw new IOException("The layout version " + logVersion throw new IOException("The layout version " + logVersion
+ " supports inodeId but gave bogus inodeId"); + " supports inodeId but gave bogus inodeId");
} }
@ -285,7 +312,7 @@ private long getAndUpdateLastInodeId(long inodeIdFromOp, int logVersion,
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
int logVersion, long lastInodeId) throws IOException { StartupOption startOpt, int logVersion, long lastInodeId) throws IOException {
long inodeId = INodeId.GRANDFATHER_INODE_ID; long inodeId = INodeId.GRANDFATHER_INODE_ID;
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("replaying edit log: " + op); LOG.trace("replaying edit log: " + op);
@ -693,6 +720,30 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId); fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId);
break; break;
} }
case OP_ROLLING_UPGRADE_START: {
if (startOpt == StartupOption.ROLLINGUPGRADE) {
final RollingUpgradeStartupOption rollingUpgradeOpt
= startOpt.getRollingUpgradeStartupOption();
if (rollingUpgradeOpt == RollingUpgradeStartupOption.ROLLBACK) {
throw new RollingUpgradeOp.RollbackException();
} else if (rollingUpgradeOpt == RollingUpgradeStartupOption.DOWNGRADE) {
//ignore upgrade marker
break;
}
}
// start rolling upgrade
final long startTime = ((RollingUpgradeOp) op).getTime();
fsNamesys.startRollingUpgradeInternal(startTime);
fsNamesys.triggerRollbackCheckpoint();
break;
}
case OP_ROLLING_UPGRADE_FINALIZE: {
final long finalizeTime = ((RollingUpgradeOp) op).getTime();
fsNamesys.finalizeRollingUpgradeInternal(finalizeTime);
fsNamesys.getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK,
NameNodeFile.IMAGE);
break;
}
case OP_ADD_CACHE_DIRECTIVE: { case OP_ADD_CACHE_DIRECTIVE: {
AddCacheDirectiveInfoOp addOp = (AddCacheDirectiveInfoOp) op; AddCacheDirectiveInfoOp addOp = (AddCacheDirectiveInfoOp) op;
CacheDirectiveInfo result = fsNamesys. CacheDirectiveInfo result = fsNamesys.
@ -931,7 +982,7 @@ private void check203UpgradeFailure(int logVersion, Throwable e)
// The editlog must be emptied by restarting the namenode, before proceeding // The editlog must be emptied by restarting the namenode, before proceeding
// with the upgrade. // with the upgrade.
if (Storage.is203LayoutVersion(logVersion) if (Storage.is203LayoutVersion(logVersion)
&& logVersion != HdfsConstants.LAYOUT_VERSION) { && logVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
String msg = "During upgrade failed to load the editlog version " String msg = "During upgrade failed to load the editlog version "
+ logVersion + " from release 0.20.203. Please go back to the old " + logVersion + " from release 0.20.203. Please go back to the old "
+ " release and restart the namenode. This empties the editlog " + " release and restart the namenode. This empties the editlog "

View File

@ -45,6 +45,8 @@
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_SNAPSHOT; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_SNAPSHOT;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENEW_DELEGATION_TOKEN; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENEW_DELEGATION_TOKEN;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_ACL; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_ACL;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ROLLING_UPGRADE_FINALIZE;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ROLLING_UPGRADE_START;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_GENSTAMP_V1; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_GENSTAMP_V1;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_GENSTAMP_V2; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_GENSTAMP_V2;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_NS_QUOTA; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_NS_QUOTA;
@ -179,7 +181,12 @@ public OpInstanceCache() {
inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp()); inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp()); inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp()); inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
inst.put(OP_SET_ACL, new SetAclOp()); inst.put(OP_SET_ACL, new SetAclOp());
inst.put(OP_ROLLING_UPGRADE_START, new RollingUpgradeOp(
OP_ROLLING_UPGRADE_START, "start"));
inst.put(OP_ROLLING_UPGRADE_FINALIZE, new RollingUpgradeOp(
OP_ROLLING_UPGRADE_FINALIZE, "finalize"));
} }
public FSEditLogOp get(FSEditLogOpCodes opcode) { public FSEditLogOp get(FSEditLogOpCodes opcode) {
@ -267,8 +274,8 @@ private static void writeRpcIds(final byte[] clientId, final int callId,
void readRpcIds(DataInputStream in, int logVersion) void readRpcIds(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (LayoutVersion.supports(Feature.EDITLOG_SUPPORT_RETRYCACHE, if (NameNodeLayoutVersion.supports(
logVersion)) { LayoutVersion.Feature.EDITLOG_SUPPORT_RETRYCACHE, logVersion)) {
this.rpcClientId = FSImageSerialization.readBytes(in); this.rpcClientId = FSImageSerialization.readBytes(in);
this.rpcCallId = FSImageSerialization.readInt(in); this.rpcCallId = FSImageSerialization.readInt(in);
} }
@ -315,7 +322,7 @@ private static final class AclEditLogUtil {
private static List<AclEntry> read(DataInputStream in, int logVersion) private static List<AclEntry> read(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EXTENDED_ACL, logVersion)) { if (!NameNodeLayoutVersion.supports(Feature.EXTENDED_ACL, logVersion)) {
return null; return null;
} }
@ -480,18 +487,20 @@ public void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
} }
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
this.inodeId = in.readLong(); this.inodeId = in.readLong();
} else { } else {
// The inodeId should be updated when this editLogOp is applied // The inodeId should be updated when this editLogOp is applied
this.inodeId = INodeId.GRANDFATHER_INODE_ID; this.inodeId = INodeId.GRANDFATHER_INODE_ID;
} }
if ((-17 < logVersion && length != 4) || if ((-17 < logVersion && length != 4) ||
(logVersion <= -17 && length != 5 && !LayoutVersion.supports( (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) { LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
throw new IOException("Incorrect data format." + throw new IOException("Incorrect data format." +
" logVersion is " + logVersion + " logVersion is " + logVersion +
" but writables.length is " + " but writables.length is " +
@ -499,7 +508,8 @@ void readFields(DataInputStream in, int logVersion)
} }
this.path = FSImageSerialization.readString(in); this.path = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.replication = FSImageSerialization.readShort(in); this.replication = FSImageSerialization.readShort(in);
this.mtime = FSImageSerialization.readLong(in); this.mtime = FSImageSerialization.readLong(in);
} else { } else {
@ -507,8 +517,10 @@ void readFields(DataInputStream in, int logVersion)
this.mtime = readLong(in); this.mtime = readLong(in);
} }
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) { if (NameNodeLayoutVersion.supports(
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.atime = FSImageSerialization.readLong(in); this.atime = FSImageSerialization.readLong(in);
} else { } else {
this.atime = readLong(in); this.atime = readLong(in);
@ -517,7 +529,8 @@ void readFields(DataInputStream in, int logVersion)
this.atime = 0; this.atime = 0;
} }
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.blockSize = FSImageSerialization.readLong(in); this.blockSize = FSImageSerialization.readLong(in);
} else { } else {
this.blockSize = readLong(in); this.blockSize = readLong(in);
@ -933,7 +946,8 @@ void writeFields(DataOutputStream out) throws IOException {
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
this.path = FSImageSerialization.readString(in); this.path = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.replication = FSImageSerialization.readShort(in); this.replication = FSImageSerialization.readShort(in);
} else { } else {
this.replication = readShort(in); this.replication = readShort(in);
@ -1024,7 +1038,8 @@ public void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
if (length < 3) { // trg, srcs.., timestamp if (length < 3) { // trg, srcs.., timestamp
throw new IOException("Incorrect data format " + throw new IOException("Incorrect data format " +
@ -1033,7 +1048,8 @@ void readFields(DataInputStream in, int logVersion)
} }
this.trg = FSImageSerialization.readString(in); this.trg = FSImageSerialization.readString(in);
int srcSize = 0; int srcSize = 0;
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
srcSize = in.readInt(); srcSize = in.readInt();
} else { } else {
srcSize = this.length - 1 - 1; // trg and timestamp srcSize = this.length - 1 - 1; // trg and timestamp
@ -1052,7 +1068,8 @@ void readFields(DataInputStream in, int logVersion)
srcs[i]= FSImageSerialization.readString(in); srcs[i]= FSImageSerialization.readString(in);
} }
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.timestamp = FSImageSerialization.readLong(in); this.timestamp = FSImageSerialization.readLong(in);
} else { } else {
this.timestamp = readLong(in); this.timestamp = readLong(in);
@ -1158,7 +1175,8 @@ void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
if (this.length != 3) { if (this.length != 3) {
throw new IOException("Incorrect data format. " throw new IOException("Incorrect data format. "
@ -1167,7 +1185,8 @@ void readFields(DataInputStream in, int logVersion)
} }
this.src = FSImageSerialization.readString(in); this.src = FSImageSerialization.readString(in);
this.dst = FSImageSerialization.readString(in); this.dst = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.timestamp = FSImageSerialization.readLong(in); this.timestamp = FSImageSerialization.readLong(in);
} else { } else {
this.timestamp = readLong(in); this.timestamp = readLong(in);
@ -1254,14 +1273,16 @@ void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
if (this.length != 2) { if (this.length != 2) {
throw new IOException("Incorrect data format. " + "delete operation."); throw new IOException("Incorrect data format. " + "delete operation.");
} }
} }
this.path = FSImageSerialization.readString(in); this.path = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.timestamp = FSImageSerialization.readLong(in); this.timestamp = FSImageSerialization.readLong(in);
} else { } else {
this.timestamp = readLong(in); this.timestamp = readLong(in);
@ -1362,22 +1383,26 @@ void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) throws IOException { void readFields(DataInputStream in, int logVersion) throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
} }
if (-17 < logVersion && length != 2 || if (-17 < logVersion && length != 2 ||
logVersion <= -17 && length != 3 logVersion <= -17 && length != 3
&& !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { && !NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
throw new IOException("Incorrect data format. Mkdir operation."); throw new IOException("Incorrect data format. Mkdir operation.");
} }
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
this.inodeId = FSImageSerialization.readLong(in); this.inodeId = FSImageSerialization.readLong(in);
} else { } else {
// This id should be updated when this editLogOp is applied // This id should be updated when this editLogOp is applied
this.inodeId = INodeId.GRANDFATHER_INODE_ID; this.inodeId = INodeId.GRANDFATHER_INODE_ID;
} }
this.path = FSImageSerialization.readString(in); this.path = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.timestamp = FSImageSerialization.readLong(in); this.timestamp = FSImageSerialization.readLong(in);
} else { } else {
this.timestamp = readLong(in); this.timestamp = readLong(in);
@ -1386,8 +1411,10 @@ void readFields(DataInputStream in, int logVersion) throws IOException {
// The disk format stores atimes for directories as well. // The disk format stores atimes for directories as well.
// However, currently this is not being updated/used because of // However, currently this is not being updated/used because of
// performance reasons. // performance reasons.
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) { if (NameNodeLayoutVersion.supports(
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
FSImageSerialization.readLong(in); FSImageSerialization.readLong(in);
} else { } else {
readLong(in); readLong(in);
@ -1977,7 +2004,8 @@ void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
if (length != 3) { if (length != 3) {
throw new IOException("Incorrect data format. " + "times operation."); throw new IOException("Incorrect data format. " + "times operation.");
@ -1985,7 +2013,8 @@ void readFields(DataInputStream in, int logVersion)
} }
this.path = FSImageSerialization.readString(in); this.path = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.mtime = FSImageSerialization.readLong(in); this.mtime = FSImageSerialization.readLong(in);
this.atime = FSImageSerialization.readLong(in); this.atime = FSImageSerialization.readLong(in);
} else { } else {
@ -2094,14 +2123,16 @@ public void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
if (this.length != 4) { if (this.length != 4) {
throw new IOException("Incorrect data format. " throw new IOException("Incorrect data format. "
+ "symlink operation."); + "symlink operation.");
} }
} }
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
this.inodeId = FSImageSerialization.readLong(in); this.inodeId = FSImageSerialization.readLong(in);
} else { } else {
// This id should be updated when the editLogOp is applied // This id should be updated when the editLogOp is applied
@ -2110,7 +2141,8 @@ void readFields(DataInputStream in, int logVersion)
this.path = FSImageSerialization.readString(in); this.path = FSImageSerialization.readString(in);
this.value = FSImageSerialization.readString(in); this.value = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.mtime = FSImageSerialization.readLong(in); this.mtime = FSImageSerialization.readLong(in);
this.atime = FSImageSerialization.readLong(in); this.atime = FSImageSerialization.readLong(in);
} else { } else {
@ -2228,7 +2260,8 @@ void writeFields(DataOutputStream out) throws IOException {
@Override @Override
void readFields(DataInputStream in, int logVersion) void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.length = in.readInt(); this.length = in.readInt();
if (this.length != 3) { if (this.length != 3) {
throw new IOException("Incorrect data format. " + "Rename operation."); throw new IOException("Incorrect data format. " + "Rename operation.");
@ -2237,7 +2270,8 @@ void readFields(DataInputStream in, int logVersion)
this.src = FSImageSerialization.readString(in); this.src = FSImageSerialization.readString(in);
this.dst = FSImageSerialization.readString(in); this.dst = FSImageSerialization.readString(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.timestamp = FSImageSerialization.readLong(in); this.timestamp = FSImageSerialization.readLong(in);
} else { } else {
this.timestamp = readLong(in); this.timestamp = readLong(in);
@ -2448,7 +2482,8 @@ void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
this.token = new DelegationTokenIdentifier(); this.token = new DelegationTokenIdentifier();
this.token.readFields(in); this.token.readFields(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.expiryTime = FSImageSerialization.readLong(in); this.expiryTime = FSImageSerialization.readLong(in);
} else { } else {
this.expiryTime = readLong(in); this.expiryTime = readLong(in);
@ -2520,7 +2555,8 @@ void readFields(DataInputStream in, int logVersion)
throws IOException { throws IOException {
this.token = new DelegationTokenIdentifier(); this.token = new DelegationTokenIdentifier();
this.token.readFields(in); this.token.readFields(in);
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.expiryTime = FSImageSerialization.readLong(in); this.expiryTime = FSImageSerialization.readLong(in);
} else { } else {
this.expiryTime = readLong(in); this.expiryTime = readLong(in);
@ -3544,6 +3580,65 @@ public void readFields(DataInput in) throws IOException {
this.len = in.readLong(); this.len = in.readLong();
} }
} }
/**
* Operation corresponding to upgrade
*/
static class RollingUpgradeOp extends FSEditLogOp { // @Idempotent
private final String name;
private long time;
public RollingUpgradeOp(FSEditLogOpCodes code, String name) {
super(code);
this.name = name.toUpperCase();
}
static RollingUpgradeOp getStartInstance(OpInstanceCache cache) {
return (RollingUpgradeOp) cache.get(OP_ROLLING_UPGRADE_START);
}
static RollingUpgradeOp getFinalizeInstance(OpInstanceCache cache) {
return (RollingUpgradeOp) cache.get(OP_ROLLING_UPGRADE_FINALIZE);
}
long getTime() {
return time;
}
void setTime(long time) {
this.time = time;
}
@Override
void readFields(DataInputStream in, int logVersion) throws IOException {
time = in.readLong();
}
@Override
public void writeFields(DataOutputStream out) throws IOException {
FSImageSerialization.writeLong(time, out);
}
@Override
protected void toXml(ContentHandler contentHandler) throws SAXException {
XMLUtils.addSaxString(contentHandler, name + "TIME",
Long.valueOf(time).toString());
}
@Override
void fromXml(Stanza st) throws InvalidXmlException {
this.time = Long.valueOf(st.getValue(name + "TIME"));
}
@Override
public String toString() {
return new StringBuilder().append("RollingUpgradeOp [").append(name)
.append(", time=").append(time).append("]").toString();
}
static class RollbackException extends IOException {
private static final long serialVersionUID = 1L;
}
}
/** /**
* Class for writing editlog ops * Class for writing editlog ops
@ -3594,7 +3689,8 @@ public static class Reader {
*/ */
public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) { public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
this.logVersion = logVersion; this.logVersion = logVersion;
if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) {
this.checksum = new PureJavaCrc32(); this.checksum = new PureJavaCrc32();
} else { } else {
this.checksum = null; this.checksum = null;
@ -3733,7 +3829,8 @@ private FSEditLogOp decodeOp() throws IOException {
throw new IOException("Read invalid opcode " + opCode); throw new IOException("Read invalid opcode " + opCode);
} }
if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
// Read the txid // Read the txid
op.setTransactionId(in.readLong()); op.setTransactionId(in.readLong());
} else { } else {

View File

@ -66,8 +66,10 @@ public enum FSEditLogOpCodes {
OP_ADD_CACHE_POOL ((byte) 36), OP_ADD_CACHE_POOL ((byte) 36),
OP_MODIFY_CACHE_POOL ((byte) 37), OP_MODIFY_CACHE_POOL ((byte) 37),
OP_REMOVE_CACHE_POOL ((byte) 38), OP_REMOVE_CACHE_POOL ((byte) 38),
OP_MODIFY_CACHE_DIRECTIVE ((byte) 39), OP_MODIFY_CACHE_DIRECTIVE ((byte) 39),
OP_SET_ACL ((byte) 40), OP_SET_ACL ((byte) 40),
OP_ROLLING_UPGRADE_START ((byte) 41),
OP_ROLLING_UPGRADE_FINALIZE ((byte) 42),
// Note that the current range of the valid OP code is 0~127 // Note that the current range of the valid OP code is 0~127
OP_INVALID ((byte) -1); OP_INVALID ((byte) -1);

View File

@ -21,11 +21,13 @@
import java.io.Closeable; import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -41,18 +43,15 @@
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException; import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageState; import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
import org.apache.hadoop.hdfs.server.common.Util; import org.apache.hadoop.hdfs.server.common.Util;
import static org.apache.hadoop.util.Time.now;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile; import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@ -218,13 +217,18 @@ boolean recoverTransitionRead(StartupOption startOpt, FSNamesystem target,
NNStorage.checkVersionUpgradable(storage.getLayoutVersion()); NNStorage.checkVersionUpgradable(storage.getLayoutVersion());
} }
if (startOpt != StartupOption.UPGRADE if (startOpt != StartupOption.UPGRADE
&& !RollingUpgradeStartupOption.STARTED.matches(startOpt)
&& layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION && layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
&& layoutVersion != HdfsConstants.LAYOUT_VERSION) { && layoutVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
throw new IOException( throw new IOException(
"\nFile system image contains an old layout version " "\nFile system image contains an old layout version "
+ storage.getLayoutVersion() + ".\nAn upgrade to version " + storage.getLayoutVersion() + ".\nAn upgrade to version "
+ HdfsConstants.LAYOUT_VERSION + " is required.\n" + HdfsConstants.NAMENODE_LAYOUT_VERSION + " is required.\n"
+ "Please restart NameNode with -upgrade option."); + "Please restart NameNode with the \""
+ RollingUpgradeStartupOption.STARTED.getOptionString()
+ "\" option if a rolling upgraded is already started;"
+ " or restart NameNode with the \""
+ StartupOption.UPGRADE + "\" to start a new upgrade.");
} }
storage.processStartupOptionsForUpgrade(startOpt, layoutVersion); storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
@ -263,7 +267,7 @@ boolean recoverTransitionRead(StartupOption startOpt, FSNamesystem target,
// just load the image // just load the image
} }
return loadFSImage(target, recovery); return loadFSImage(target, startOpt, recovery);
} }
/** /**
@ -319,8 +323,9 @@ private boolean recoverStorageDirs(StartupOption startOpt,
return isFormatted; return isFormatted;
} }
private void doUpgrade(FSNamesystem target) throws IOException { /** Check if upgrade is in progress. */
// Upgrade is allowed only if there are void checkUpgrade(FSNamesystem target) throws IOException {
// Upgrade or rolling upgrade is allowed only if there are
// no previous fs states in any of the directories // no previous fs states in any of the directories
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) { for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next(); StorageDirectory sd = it.next();
@ -329,15 +334,37 @@ private void doUpgrade(FSNamesystem target) throws IOException {
"previous fs state should not exist during upgrade. " "previous fs state should not exist during upgrade. "
+ "Finalize or rollback first."); + "Finalize or rollback first.");
} }
}
/**
* @return true if there is rollback fsimage (for rolling upgrade) in NameNode
* directory.
*/
public boolean hasRollbackFSImage() throws IOException {
final FSImageStorageInspector inspector = new FSImageTransactionalStorageInspector(
EnumSet.of(NameNodeFile.IMAGE_ROLLBACK));
storage.inspectStorageDirs(inspector);
try {
List<FSImageFile> images = inspector.getLatestImages();
return images != null && !images.isEmpty();
} catch (FileNotFoundException e) {
return false;
}
}
void doUpgrade(FSNamesystem target) throws IOException {
checkUpgrade(target);
// load the latest image // load the latest image
this.loadFSImage(target, null); this.loadFSImage(target, null, null);
// Do upgrade for each directory // Do upgrade for each directory
target.checkRollingUpgrade("upgrade namenode");
long oldCTime = storage.getCTime(); long oldCTime = storage.getCTime();
storage.cTime = now(); // generate new cTime for the state storage.cTime = now(); // generate new cTime for the state
int oldLV = storage.getLayoutVersion(); int oldLV = storage.getLayoutVersion();
storage.layoutVersion = HdfsConstants.LAYOUT_VERSION; storage.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
List<StorageDirectory> errorSDs = List<StorageDirectory> errorSDs =
Collections.synchronizedList(new ArrayList<StorageDirectory>()); Collections.synchronizedList(new ArrayList<StorageDirectory>());
@ -411,7 +438,7 @@ private void doRollback() throws IOException {
boolean canRollback = false; boolean canRollback = false;
FSImage prevState = new FSImage(conf); FSImage prevState = new FSImage(conf);
try { try {
prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION; prevState.getStorage().layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) { for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next(); StorageDirectory sd = it.next();
File prevDir = sd.getPreviousDir(); File prevDir = sd.getPreviousDir();
@ -426,12 +453,12 @@ private void doRollback() throws IOException {
// read and verify consistency of the prev dir // read and verify consistency of the prev dir
prevState.getStorage().readPreviousVersionProperties(sd); prevState.getStorage().readPreviousVersionProperties(sd);
if (prevState.getLayoutVersion() != HdfsConstants.LAYOUT_VERSION) { if (prevState.getLayoutVersion() != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
throw new IOException( throw new IOException(
"Cannot rollback to storage version " + "Cannot rollback to storage version " +
prevState.getLayoutVersion() + prevState.getLayoutVersion() +
" using this version of the NameNode, which uses storage version " + " using this version of the NameNode, which uses storage version " +
HdfsConstants.LAYOUT_VERSION + ". " + HdfsConstants.NAMENODE_LAYOUT_VERSION + ". " +
"Please use the previous version of HDFS to perform the rollback."); "Please use the previous version of HDFS to perform the rollback.");
} }
canRollback = true; canRollback = true;
@ -525,7 +552,7 @@ void doImportCheckpoint(FSNamesystem target) throws IOException {
// return back the real image // return back the real image
realImage.getStorage().setStorageInfo(ckptImage.getStorage()); realImage.getStorage().setStorageInfo(ckptImage.getStorage());
realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1); realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
realImage.initEditLog(); realImage.initEditLog(StartupOption.IMPORT);
target.dir.fsImage = realImage; target.dir.fsImage = realImage;
realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID()); realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
@ -586,13 +613,22 @@ void reloadFromImageFile(File file, FSNamesystem target) throws IOException {
* @return whether the image should be saved * @return whether the image should be saved
* @throws IOException * @throws IOException
*/ */
boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery) private boolean loadFSImage(FSNamesystem target, StartupOption startOpt,
MetaRecoveryContext recovery)
throws IOException { throws IOException {
FSImageStorageInspector inspector = storage.readAndInspectDirs(); final boolean rollingRollback
FSImageFile imageFile = null; = RollingUpgradeStartupOption.ROLLBACK.matches(startOpt);
final EnumSet<NameNodeFile> nnfs;
if (rollingRollback) {
// if it is rollback of rolling upgrade, only load from the rollback image
nnfs = EnumSet.of(NameNodeFile.IMAGE_ROLLBACK);
} else {
// otherwise we can load from both IMAGE and IMAGE_ROLLBACK
nnfs = EnumSet.of(NameNodeFile.IMAGE, NameNodeFile.IMAGE_ROLLBACK);
}
final FSImageStorageInspector inspector = storage.readAndInspectDirs(nnfs);
isUpgradeFinalized = inspector.isUpgradeFinalized(); isUpgradeFinalized = inspector.isUpgradeFinalized();
List<FSImageFile> imageFiles = inspector.getLatestImages(); List<FSImageFile> imageFiles = inspector.getLatestImages();
StartupProgress prog = NameNode.getStartupProgress(); StartupProgress prog = NameNode.getStartupProgress();
@ -604,14 +640,24 @@ boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery)
Iterable<EditLogInputStream> editStreams = null; Iterable<EditLogInputStream> editStreams = null;
initEditLog(); initEditLog(startOpt);
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, if (NameNodeLayoutVersion.supports(
getLayoutVersion())) { LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
// If we're open for write, we're either non-HA or we're the active NN, so // If we're open for write, we're either non-HA or we're the active NN, so
// we better be able to load all the edits. If we're the standby NN, it's // we better be able to load all the edits. If we're the standby NN, it's
// OK to not be able to read all of edits right now. // OK to not be able to read all of edits right now.
long toAtLeastTxId = editLog.isOpenForWrite() ? inspector.getMaxSeenTxId() : 0; // In the meanwhile, for HA upgrade, we will still write editlog thus need
// this toAtLeastTxId to be set to the max-seen txid
// For rollback in rolling upgrade, we need to set the toAtLeastTxId to
// the txid right before the upgrade marker.
long toAtLeastTxId = editLog.isOpenForWrite() ? inspector
.getMaxSeenTxId() : 0;
if (rollingRollback) {
// note that the first image in imageFiles is the special checkpoint
// for the rolling upgrade
toAtLeastTxId = imageFiles.get(0).getCheckpointTxId() + 2;
}
editStreams = editLog.selectInputStreams( editStreams = editLog.selectInputStreams(
imageFiles.get(0).getCheckpointTxId() + 1, imageFiles.get(0).getCheckpointTxId() + 1,
toAtLeastTxId, recovery, false); toAtLeastTxId, recovery, false);
@ -619,8 +665,7 @@ boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery)
editStreams = FSImagePreTransactionalStorageInspector editStreams = FSImagePreTransactionalStorageInspector
.getEditLogStreams(storage); .getEditLogStreams(storage);
} }
int maxOpSize = conf.getInt(DFSConfigKeys. int maxOpSize = conf.getInt(DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_KEY,
DFS_NAMENODE_MAX_OP_SIZE_KEY,
DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT); DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT);
for (EditLogInputStream elis : editStreams) { for (EditLogInputStream elis : editStreams) {
elis.setMaxOpSize(maxOpSize); elis.setMaxOpSize(maxOpSize);
@ -633,6 +678,7 @@ boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery)
LOG.info("No edit log streams selected."); LOG.info("No edit log streams selected.");
} }
FSImageFile imageFile = null;
for (int i = 0; i < imageFiles.size(); i++) { for (int i = 0; i < imageFiles.size(); i++) {
try { try {
imageFile = imageFiles.get(i); imageFile = imageFiles.get(i);
@ -650,26 +696,57 @@ boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery)
throw new IOException("Failed to load an FSImage file!"); throw new IOException("Failed to load an FSImage file!");
} }
prog.endPhase(Phase.LOADING_FSIMAGE); prog.endPhase(Phase.LOADING_FSIMAGE);
long txnsAdvanced = loadEdits(editStreams, target, recovery);
needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(), if (!rollingRollback) {
txnsAdvanced); long txnsAdvanced = loadEdits(editStreams, target, startOpt, recovery);
needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(),
txnsAdvanced);
if (RollingUpgradeStartupOption.DOWNGRADE.matches(startOpt)) {
// rename rollback image if it is downgrade
renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK, NameNodeFile.IMAGE);
}
} else {
// Trigger the rollback for rolling upgrade. Here lastAppliedTxId equals
// to the last txid in rollback fsimage.
rollingRollback(lastAppliedTxId + 1, imageFiles.get(0).getCheckpointTxId());
needToSave = false;
}
editLog.setNextTxId(lastAppliedTxId + 1); editLog.setNextTxId(lastAppliedTxId + 1);
return needToSave; return needToSave;
} }
/** rollback for rolling upgrade. */
private void rollingRollback(long discardSegmentTxId, long ckptId)
throws IOException {
// discard discard unnecessary editlog segments starting from the given id
this.editLog.discardSegments(discardSegmentTxId);
// rename the special checkpoint
renameCheckpoint(ckptId, NameNodeFile.IMAGE_ROLLBACK, NameNodeFile.IMAGE,
true);
// purge all the checkpoints after the marker
archivalManager.purgeCheckpoinsAfter(NameNodeFile.IMAGE, ckptId);
String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
if (HAUtil.isHAEnabled(conf, nameserviceId)) {
// close the editlog since it is currently open for write
this.editLog.close();
// reopen the editlog for read
this.editLog.initSharedJournalsForRead();
}
}
void loadFSImageFile(FSNamesystem target, MetaRecoveryContext recovery, void loadFSImageFile(FSNamesystem target, MetaRecoveryContext recovery,
FSImageFile imageFile) throws IOException { FSImageFile imageFile) throws IOException {
LOG.debug("Planning to load image :\n" + imageFile); LOG.debug("Planning to load image :\n" + imageFile);
StorageDirectory sdForProperties = imageFile.sd; StorageDirectory sdForProperties = imageFile.sd;
storage.readProperties(sdForProperties); storage.readProperties(sdForProperties);
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, if (NameNodeLayoutVersion.supports(
getLayoutVersion())) { LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
// For txid-based layout, we should have a .md5 file // For txid-based layout, we should have a .md5 file
// next to the image file // next to the image file
loadFSImage(imageFile.getFile(), target, recovery); loadFSImage(imageFile.getFile(), target, recovery);
} else if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM, } else if (NameNodeLayoutVersion.supports(
getLayoutVersion())) { LayoutVersion.Feature.FSIMAGE_CHECKSUM, getLayoutVersion())) {
// In 0.22, we have the checksum stored in the VERSION file. // In 0.22, we have the checksum stored in the VERSION file.
String md5 = storage.getDeprecatedProperty( String md5 = storage.getDeprecatedProperty(
NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY); NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY);
@ -686,11 +763,15 @@ void loadFSImageFile(FSNamesystem target, MetaRecoveryContext recovery,
} }
} }
public void initEditLog() { public void initEditLog(StartupOption startOpt) {
Preconditions.checkState(getNamespaceID() != 0, Preconditions.checkState(getNamespaceID() != 0,
"Must know namespace ID before initting edit log"); "Must know namespace ID before initting edit log");
String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf); String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
if (!HAUtil.isHAEnabled(conf, nameserviceId)) { if (!HAUtil.isHAEnabled(conf, nameserviceId) ||
(HAUtil.isHAEnabled(conf, nameserviceId) &&
RollingUpgradeStartupOption.ROLLBACK.matches(startOpt))) {
// If this NN is not HA or this NN is HA, but we're doing a rollback of
// rolling upgrade so init the edit log for write.
editLog.initJournalsForWrite(); editLog.initJournalsForWrite();
editLog.recoverUnclosedStreams(); editLog.recoverUnclosedStreams();
} else { } else {
@ -722,7 +803,13 @@ private boolean needsResaveBasedOnStaleCheckpoint(
* Load the specified list of edit files into the image. * Load the specified list of edit files into the image.
*/ */
public long loadEdits(Iterable<EditLogInputStream> editStreams, public long loadEdits(Iterable<EditLogInputStream> editStreams,
FSNamesystem target, MetaRecoveryContext recovery) throws IOException { FSNamesystem target) throws IOException {
return loadEdits(editStreams, target, null, null);
}
private long loadEdits(Iterable<EditLogInputStream> editStreams,
FSNamesystem target, StartupOption startOpt, MetaRecoveryContext recovery)
throws IOException {
LOG.debug("About to load edits:\n " + Joiner.on("\n ").join(editStreams)); LOG.debug("About to load edits:\n " + Joiner.on("\n ").join(editStreams));
StartupProgress prog = NameNode.getStartupProgress(); StartupProgress prog = NameNode.getStartupProgress();
prog.beginPhase(Phase.LOADING_EDITS); prog.beginPhase(Phase.LOADING_EDITS);
@ -736,7 +823,7 @@ public long loadEdits(Iterable<EditLogInputStream> editStreams,
LOG.info("Reading " + editIn + " expecting start txid #" + LOG.info("Reading " + editIn + " expecting start txid #" +
(lastAppliedTxId + 1)); (lastAppliedTxId + 1));
try { try {
loader.loadFSEdits(editIn, lastAppliedTxId + 1, recovery); loader.loadFSEdits(editIn, lastAppliedTxId + 1, startOpt, recovery);
} finally { } finally {
// Update lastAppliedTxId even in case of error, since some ops may // Update lastAppliedTxId even in case of error, since some ops may
// have been successfully applied before the error. // have been successfully applied before the error.
@ -750,7 +837,7 @@ public long loadEdits(Iterable<EditLogInputStream> editStreams,
} finally { } finally {
FSEditLog.closeAllStreams(editStreams); FSEditLog.closeAllStreams(editStreams);
// update the counts // update the counts
updateCountForQuota(target.dir.rootDir); updateCountForQuota(target.dir.rootDir);
} }
prog.endPhase(Phase.LOADING_EDITS); prog.endPhase(Phase.LOADING_EDITS);
return lastAppliedTxId - prevLastAppliedTxId; return lastAppliedTxId - prevLastAppliedTxId;
@ -828,9 +915,12 @@ private void loadFSImage(File imageFile, FSNamesystem target,
*/ */
private void loadFSImage(File curFile, MD5Hash expectedMd5, private void loadFSImage(File curFile, MD5Hash expectedMd5,
FSNamesystem target, MetaRecoveryContext recovery) throws IOException { FSNamesystem target, MetaRecoveryContext recovery) throws IOException {
// BlockPoolId is required when the FsImageLoader loads the rolling upgrade
// information. Make sure the ID is properly set.
target.setBlockPoolId(this.getBlockPoolID());
FSImageFormat.LoaderDelegator loader = FSImageFormat.newLoader(conf, target); FSImageFormat.LoaderDelegator loader = FSImageFormat.newLoader(conf, target);
loader.load(curFile); loader.load(curFile);
target.setBlockPoolId(this.getBlockPoolID());
// Check that the image digest we loaded matches up with what // Check that the image digest we loaded matches up with what
// we expected // we expected
@ -851,11 +941,11 @@ private void loadFSImage(File curFile, MD5Hash expectedMd5,
/** /**
* Save the contents of the FS image to the file. * Save the contents of the FS image to the file.
*/ */
void saveFSImage(SaveNamespaceContext context, StorageDirectory sd) void saveFSImage(SaveNamespaceContext context, StorageDirectory sd,
throws IOException { NameNodeFile dstType) throws IOException {
long txid = context.getTxId(); long txid = context.getTxId();
File newFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid); File newFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
File dstFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid); File dstFile = NNStorage.getStorageFile(sd, dstType, txid);
FSImageFormatProtobuf.Saver saver = new FSImageFormatProtobuf.Saver(context); FSImageFormatProtobuf.Saver saver = new FSImageFormatProtobuf.Saver(context);
FSImageCompression compression = FSImageCompression.createCompression(conf); FSImageCompression compression = FSImageCompression.createCompression(conf);
@ -879,16 +969,19 @@ void saveFSImage(SaveNamespaceContext context, StorageDirectory sd)
private class FSImageSaver implements Runnable { private class FSImageSaver implements Runnable {
private final SaveNamespaceContext context; private final SaveNamespaceContext context;
private StorageDirectory sd; private StorageDirectory sd;
private final NameNodeFile nnf;
public FSImageSaver(SaveNamespaceContext context, StorageDirectory sd) { public FSImageSaver(SaveNamespaceContext context, StorageDirectory sd,
NameNodeFile nnf) {
this.context = context; this.context = context;
this.sd = sd; this.sd = sd;
this.nnf = nnf;
} }
@Override @Override
public void run() { public void run() {
try { try {
saveFSImage(context, sd); saveFSImage(context, sd, nnf);
} catch (SaveNamespaceCancelledException snce) { } catch (SaveNamespaceCancelledException snce) {
LOG.info("Cancelled image saving for " + sd.getRoot() + LOG.info("Cancelled image saving for " + sd.getRoot() +
": " + snce.getMessage()); ": " + snce.getMessage());
@ -924,17 +1017,18 @@ private void waitForThreads(List<Thread> threads) {
*/ */
public synchronized void saveNamespace(FSNamesystem source) public synchronized void saveNamespace(FSNamesystem source)
throws IOException { throws IOException {
saveNamespace(source, null); saveNamespace(source, NameNodeFile.IMAGE, null);
} }
/** /**
* Save the contents of the FS image to a new image file in each of the * Save the contents of the FS image to a new image file in each of the
* current storage directories. * current storage directories.
* @param canceler * @param canceler
*/ */
public synchronized void saveNamespace(FSNamesystem source, public synchronized void saveNamespace(FSNamesystem source, NameNodeFile nnf,
Canceler canceler) throws IOException { Canceler canceler) throws IOException {
assert editLog != null : "editLog must be initialized"; assert editLog != null : "editLog must be initialized";
LOG.info("Save namespace ...");
storage.attemptRestoreRemovedStorage(); storage.attemptRestoreRemovedStorage();
boolean editLogWasOpen = editLog.isSegmentOpen(); boolean editLogWasOpen = editLog.isSegmentOpen();
@ -944,7 +1038,7 @@ public synchronized void saveNamespace(FSNamesystem source,
} }
long imageTxId = getLastAppliedOrWrittenTxId(); long imageTxId = getLastAppliedOrWrittenTxId();
try { try {
saveFSImageInAllDirs(source, imageTxId, canceler); saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
storage.writeAll(); storage.writeAll();
} finally { } finally {
if (editLogWasOpen) { if (editLogWasOpen) {
@ -963,12 +1057,11 @@ public synchronized void saveNamespace(FSNamesystem source,
*/ */
protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid) protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid)
throws IOException { throws IOException {
saveFSImageInAllDirs(source, txid, null); saveFSImageInAllDirs(source, NameNodeFile.IMAGE, txid, null);
} }
protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid, private synchronized void saveFSImageInAllDirs(FSNamesystem source,
Canceler canceler) NameNodeFile nnf, long txid, Canceler canceler) throws IOException {
throws IOException {
StartupProgress prog = NameNode.getStartupProgress(); StartupProgress prog = NameNode.getStartupProgress();
prog.beginPhase(Phase.SAVING_CHECKPOINT); prog.beginPhase(Phase.SAVING_CHECKPOINT);
if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) { if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
@ -986,7 +1079,7 @@ protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid,
for (Iterator<StorageDirectory> it for (Iterator<StorageDirectory> it
= storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) { = storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
StorageDirectory sd = it.next(); StorageDirectory sd = it.next();
FSImageSaver saver = new FSImageSaver(ctx, sd); FSImageSaver saver = new FSImageSaver(ctx, sd, nnf);
Thread saveThread = new Thread(saver, saver.toString()); Thread saveThread = new Thread(saver, saver.toString());
saveThreads.add(saveThread); saveThreads.add(saveThread);
saveThread.start(); saveThread.start();
@ -1005,11 +1098,11 @@ protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid,
assert false : "should have thrown above!"; assert false : "should have thrown above!";
} }
renameCheckpoint(txid); renameCheckpoint(txid, NameNodeFile.IMAGE_NEW, nnf, false);
// Since we now have a new checkpoint, we can clean up some // Since we now have a new checkpoint, we can clean up some
// old edit logs and checkpoints. // old edit logs and checkpoints.
purgeOldStorage(); purgeOldStorage(nnf);
} finally { } finally {
// Notify any threads waiting on the checkpoint to be canceled // Notify any threads waiting on the checkpoint to be canceled
// that it is complete. // that it is complete.
@ -1023,23 +1116,24 @@ protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid,
* Purge any files in the storage directories that are no longer * Purge any files in the storage directories that are no longer
* necessary. * necessary.
*/ */
public void purgeOldStorage() { void purgeOldStorage(NameNodeFile nnf) {
try { try {
archivalManager.purgeOldStorage(); archivalManager.purgeOldStorage(nnf);
} catch (Exception e) { } catch (Exception e) {
LOG.warn("Unable to purge old storage", e); LOG.warn("Unable to purge old storage " + nnf.getName(), e);
} }
} }
/** /**
* Renames new image * Rename FSImage with the specific txid
*/ */
private void renameCheckpoint(long txid) throws IOException { private void renameCheckpoint(long txid, NameNodeFile fromNnf,
NameNodeFile toNnf, boolean renameMD5) throws IOException {
ArrayList<StorageDirectory> al = null; ArrayList<StorageDirectory> al = null;
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) { for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
try { try {
renameCheckpointInDir(sd, txid); renameImageFileInDir(sd, fromNnf, toNnf, txid, renameMD5);
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.warn("Unable to rename checkpoint in " + sd, ioe); LOG.warn("Unable to rename checkpoint in " + sd, ioe);
if (al == null) { if (al == null) {
@ -1050,7 +1144,33 @@ private void renameCheckpoint(long txid) throws IOException {
} }
if(al != null) storage.reportErrorsOnDirectories(al); if(al != null) storage.reportErrorsOnDirectories(al);
} }
/**
* Rename all the fsimage files with the specific NameNodeFile type. The
* associated checksum files will also be renamed.
*/
void renameCheckpoint(NameNodeFile fromNnf, NameNodeFile toNnf)
throws IOException {
ArrayList<StorageDirectory> al = null;
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector(EnumSet.of(fromNnf));
storage.inspectStorageDirs(inspector);
for (FSImageFile image : inspector.getFoundImages()) {
try {
renameImageFileInDir(image.sd, fromNnf, toNnf, image.txId, true);
} catch (IOException ioe) {
LOG.warn("Unable to rename checkpoint in " + image.sd, ioe);
if (al == null) {
al = Lists.newArrayList();
}
al.add(image.sd);
}
}
if(al != null) {
storage.reportErrorsOnDirectories(al);
}
}
/** /**
* Deletes the checkpoint file in every storage directory, * Deletes the checkpoint file in every storage directory,
* since the checkpoint was cancelled. * since the checkpoint was cancelled.
@ -1068,23 +1188,24 @@ private void deleteCancelledCheckpoint(long txid) throws IOException {
storage.reportErrorsOnDirectories(al); storage.reportErrorsOnDirectories(al);
} }
private void renameImageFileInDir(StorageDirectory sd, NameNodeFile fromNnf,
private void renameCheckpointInDir(StorageDirectory sd, long txid) NameNodeFile toNnf, long txid, boolean renameMD5) throws IOException {
throws IOException { final File fromFile = NNStorage.getStorageFile(sd, fromNnf, txid);
File ckpt = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid); final File toFile = NNStorage.getStorageFile(sd, toNnf, txid);
File curFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid); // renameTo fails on Windows if the destination file already exists.
// renameTo fails on Windows if the destination file
// already exists.
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug("renaming " + ckpt.getAbsolutePath() LOG.debug("renaming " + fromFile.getAbsolutePath()
+ " to " + curFile.getAbsolutePath()); + " to " + toFile.getAbsolutePath());
} }
if (!ckpt.renameTo(curFile)) { if (!fromFile.renameTo(toFile)) {
if (!curFile.delete() || !ckpt.renameTo(curFile)) { if (!toFile.delete() || !fromFile.renameTo(toFile)) {
throw new IOException("renaming " + ckpt.getAbsolutePath() + " to " + throw new IOException("renaming " + fromFile.getAbsolutePath() + " to " +
curFile.getAbsolutePath() + " FAILED"); toFile.getAbsolutePath() + " FAILED");
} }
} }
if (renameMD5) {
MD5FileUtils.renameMD5File(fromFile, toFile);
}
} }
CheckpointSignature rollEditLog() throws IOException { CheckpointSignature rollEditLog() throws IOException {
@ -1165,13 +1286,13 @@ void endCheckpoint(CheckpointSignature sig) throws IOException {
* renames the image from fsimage_N.ckpt to fsimage_N and also * renames the image from fsimage_N.ckpt to fsimage_N and also
* saves the related .md5 file into place. * saves the related .md5 file into place.
*/ */
public synchronized void saveDigestAndRenameCheckpointImage( public synchronized void saveDigestAndRenameCheckpointImage(NameNodeFile nnf,
long txid, MD5Hash digest) throws IOException { long txid, MD5Hash digest) throws IOException {
// Write and rename MD5 file // Write and rename MD5 file
List<StorageDirectory> badSds = Lists.newArrayList(); List<StorageDirectory> badSds = Lists.newArrayList();
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) { for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
File imageFile = NNStorage.getImageFile(sd, txid); File imageFile = NNStorage.getImageFile(sd, nnf, txid);
try { try {
MD5FileUtils.saveMD5File(imageFile, digest); MD5FileUtils.saveMD5File(imageFile, digest);
} catch (IOException ioe) { } catch (IOException ioe) {
@ -1183,7 +1304,7 @@ public synchronized void saveDigestAndRenameCheckpointImage(
CheckpointFaultInjector.getInstance().afterMD5Rename(); CheckpointFaultInjector.getInstance().afterMD5Rename();
// Rename image from tmp file // Rename image from tmp file
renameCheckpoint(txid); renameCheckpoint(txid, NameNodeFile.IMAGE_NEW, nnf, false);
// So long as this is the newest image available, // So long as this is the newest image available,
// advertise it as such to other checkpointers // advertise it as such to other checkpointers
// from now on // from now on

View File

@ -329,9 +329,10 @@ public void load(File curFile) throws IOException {
"imgVersion " + imgVersion + "imgVersion " + imgVersion +
" expected to be " + getLayoutVersion()); " expected to be " + getLayoutVersion());
} }
boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT, boolean supportSnapshot = NameNodeLayoutVersion.supports(
imgVersion); LayoutVersion.Feature.SNAPSHOT, imgVersion);
if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_LAYOUT_FLAGS, imgVersion)) {
LayoutFlags.read(in); LayoutFlags.read(in);
} }
@ -344,7 +345,8 @@ public void load(File curFile) throws IOException {
long genstamp = in.readLong(); long genstamp = in.readLong();
namesystem.setGenerationStampV1(genstamp); namesystem.setGenerationStampV1(genstamp);
if (LayoutVersion.supports(Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
// read the starting generation stamp for sequential block IDs // read the starting generation stamp for sequential block IDs
genstamp = in.readLong(); genstamp = in.readLong();
namesystem.setGenerationStampV2(genstamp); namesystem.setGenerationStampV2(genstamp);
@ -366,14 +368,16 @@ public void load(File curFile) throws IOException {
// read the transaction ID of the last edit represented by // read the transaction ID of the last edit represented by
// this image // this image
if (LayoutVersion.supports(Feature.STORED_TXIDS, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.STORED_TXIDS, imgVersion)) {
imgTxId = in.readLong(); imgTxId = in.readLong();
} else { } else {
imgTxId = 0; imgTxId = 0;
} }
// read the last allocated inode id in the fsimage // read the last allocated inode id in the fsimage
if (LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, imgVersion)) {
long lastInodeId = in.readLong(); long lastInodeId = in.readLong();
namesystem.resetLastInodeId(lastInodeId); namesystem.resetLastInodeId(lastInodeId);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -392,7 +396,8 @@ public void load(File curFile) throws IOException {
// read compression related info // read compression related info
FSImageCompression compression; FSImageCompression compression;
if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FSIMAGE_COMPRESSION, imgVersion)) {
compression = FSImageCompression.readCompressionHeader(conf, in); compression = FSImageCompression.readCompressionHeader(conf, in);
} else { } else {
compression = FSImageCompression.createNoopCompression(); compression = FSImageCompression.createNoopCompression();
@ -405,8 +410,8 @@ public void load(File curFile) throws IOException {
LOG.info("Number of files = " + numFiles); LOG.info("Number of files = " + numFiles);
prog.setTotal(Phase.LOADING_FSIMAGE, step, numFiles); prog.setTotal(Phase.LOADING_FSIMAGE, step, numFiles);
Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step); Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, if (NameNodeLayoutVersion.supports(
imgVersion)) { LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, imgVersion)) {
if (supportSnapshot) { if (supportSnapshot) {
loadLocalNameINodesWithSnapshot(numFiles, in, counter); loadLocalNameINodesWithSnapshot(numFiles, in, counter);
} else { } else {
@ -463,9 +468,10 @@ private void updateRootAttr(INodeWithAdditionalFields root) {
*/ */
private void loadLocalNameINodesWithSnapshot(long numFiles, DataInput in, private void loadLocalNameINodesWithSnapshot(long numFiles, DataInput in,
Counter counter) throws IOException { Counter counter) throws IOException {
assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, assert NameNodeLayoutVersion.supports(
getLayoutVersion()); LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, getLayoutVersion());
assert LayoutVersion.supports(Feature.SNAPSHOT, getLayoutVersion()); assert NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SNAPSHOT, getLayoutVersion());
// load root // load root
loadRoot(in, counter); loadRoot(in, counter);
@ -484,8 +490,8 @@ private void loadLocalNameINodesWithSnapshot(long numFiles, DataInput in,
*/ */
private void loadLocalNameINodes(long numFiles, DataInput in, Counter counter) private void loadLocalNameINodes(long numFiles, DataInput in, Counter counter)
throws IOException { throws IOException {
assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, assert NameNodeLayoutVersion.supports(
getLayoutVersion()); LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, getLayoutVersion());
assert numFiles > 0; assert numFiles > 0;
// load root // load root
@ -712,18 +718,21 @@ public INode loadINodeWithLocalName(boolean isSnapshotINode,
INode loadINode(final byte[] localName, boolean isSnapshotINode, INode loadINode(final byte[] localName, boolean isSnapshotINode,
DataInput in, Counter counter) throws IOException { DataInput in, Counter counter) throws IOException {
final int imgVersion = getLayoutVersion(); final int imgVersion = getLayoutVersion();
if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SNAPSHOT, imgVersion)) {
namesystem.getFSDirectory().verifyINodeName(localName); namesystem.getFSDirectory().verifyINodeName(localName);
} }
long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? long inodeId = NameNodeLayoutVersion.supports(
in.readLong() : namesystem.allocateNewInodeId(); LayoutVersion.Feature.ADD_INODE_ID, imgVersion) ? in.readLong()
: namesystem.allocateNewInodeId();
final short replication = namesystem.getBlockManager().adjustReplication( final short replication = namesystem.getBlockManager().adjustReplication(
in.readShort()); in.readShort());
final long modificationTime = in.readLong(); final long modificationTime = in.readLong();
long atime = 0; long atime = 0;
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FILE_ACCESS_TIME, imgVersion)) {
atime = in.readLong(); atime = in.readLong();
} }
final long blockSize = in.readLong(); final long blockSize = in.readLong();
@ -743,7 +752,8 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode,
String clientMachine = ""; String clientMachine = "";
boolean underConstruction = false; boolean underConstruction = false;
FileDiffList fileDiffs = null; FileDiffList fileDiffs = null;
if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SNAPSHOT, imgVersion)) {
// read diffs // read diffs
fileDiffs = SnapshotFSImageFormat.loadFileDiffList(in, this); fileDiffs = SnapshotFSImageFormat.loadFileDiffList(in, this);
@ -780,14 +790,16 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode,
//read quotas //read quotas
final long nsQuota = in.readLong(); final long nsQuota = in.readLong();
long dsQuota = -1L; long dsQuota = -1L;
if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.DISKSPACE_QUOTA, imgVersion)) {
dsQuota = in.readLong(); dsQuota = in.readLong();
} }
//read snapshot info //read snapshot info
boolean snapshottable = false; boolean snapshottable = false;
boolean withSnapshot = false; boolean withSnapshot = false;
if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SNAPSHOT, imgVersion)) {
snapshottable = in.readBoolean(); snapshottable = in.readBoolean();
if (!snapshottable) { if (!snapshottable) {
withSnapshot = in.readBoolean(); withSnapshot = in.readBoolean();
@ -853,7 +865,8 @@ public INodeFileAttributes loadINodeFileAttributes(DataInput in)
throws IOException { throws IOException {
final int layoutVersion = getLayoutVersion(); final int layoutVersion = getLayoutVersion();
if (!LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) {
return loadINodeWithLocalName(true, in, false).asFile(); return loadINodeWithLocalName(true, in, false).asFile();
} }
@ -874,7 +887,8 @@ public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)
throws IOException { throws IOException {
final int layoutVersion = getLayoutVersion(); final int layoutVersion = getLayoutVersion();
if (!LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) {
return loadINodeWithLocalName(true, in, false).asDirectory(); return loadINodeWithLocalName(true, in, false).asDirectory();
} }
@ -909,7 +923,8 @@ private void loadFilesUnderConstruction(DataInput in,
INodeFile oldnode = null; INodeFile oldnode = null;
boolean inSnapshot = false; boolean inSnapshot = false;
if (path != null && FSDirectory.isReservedName(path) && if (path != null && FSDirectory.isReservedName(path) &&
LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) { NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) {
// TODO: for HDFS-5428, we use reserved path for those INodeFileUC in // TODO: for HDFS-5428, we use reserved path for those INodeFileUC in
// snapshot. If we support INode ID in the layout version, we can use // snapshot. If we support INode ID in the layout version, we can use
// the inode id to find the oldnode. // the inode id to find the oldnode.
@ -942,7 +957,8 @@ private void loadSecretManagerState(DataInput in)
throws IOException { throws IOException {
int imgVersion = getLayoutVersion(); int imgVersion = getLayoutVersion();
if (!LayoutVersion.supports(Feature.DELEGATION_TOKEN, imgVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.DELEGATION_TOKEN, imgVersion)) {
//SecretManagerState is not available. //SecretManagerState is not available.
//This must not happen if security is turned on. //This must not happen if security is turned on.
return; return;
@ -952,7 +968,8 @@ private void loadSecretManagerState(DataInput in)
private void loadCacheManagerState(DataInput in) throws IOException { private void loadCacheManagerState(DataInput in) throws IOException {
int imgVersion = getLayoutVersion(); int imgVersion = getLayoutVersion();
if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.CACHING, imgVersion)) {
return; return;
} }
namesystem.getCacheManager().loadStateCompat(in); namesystem.getCacheManager().loadStateCompat(in);
@ -1014,7 +1031,7 @@ public static void useDefaultRenameReservedPairs() {
for (String key: HdfsConstants.RESERVED_PATH_COMPONENTS) { for (String key: HdfsConstants.RESERVED_PATH_COMPONENTS) {
renameReservedMap.put( renameReservedMap.put(
key, key,
key + "." + LayoutVersion.getCurrentLayoutVersion() + "." key + "." + HdfsConstants.NAMENODE_LAYOUT_VERSION + "."
+ "UPGRADE_RENAMED"); + "UPGRADE_RENAMED");
} }
} }
@ -1062,7 +1079,7 @@ static String renameReservedPathsOnUpgrade(String path,
final int layoutVersion) { final int layoutVersion) {
final String oldPath = path; final String oldPath = path;
// If any known LVs aren't supported, we're doing an upgrade // If any known LVs aren't supported, we're doing an upgrade
if (!LayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) { if (!NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) {
String[] components = INode.getPathNames(path); String[] components = INode.getPathNames(path);
// Only need to worry about the root directory // Only need to worry about the root directory
if (components.length > 1) { if (components.length > 1) {
@ -1073,7 +1090,7 @@ static String renameReservedPathsOnUpgrade(String path,
path = DFSUtil.strings2PathString(components); path = DFSUtil.strings2PathString(components);
} }
} }
if (!LayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) { if (!NameNodeLayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
String[] components = INode.getPathNames(path); String[] components = INode.getPathNames(path);
// Special case the root path // Special case the root path
if (components.length == 0) { if (components.length == 0) {
@ -1111,7 +1128,7 @@ static String renameReservedPathsOnUpgrade(String path,
private static byte[] renameReservedComponentOnUpgrade(byte[] component, private static byte[] renameReservedComponentOnUpgrade(byte[] component,
final int layoutVersion) { final int layoutVersion) {
// If the LV doesn't support snapshots, we're doing an upgrade // If the LV doesn't support snapshots, we're doing an upgrade
if (!LayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) { if (!NameNodeLayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
if (Arrays.equals(component, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES)) { if (Arrays.equals(component, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES)) {
Preconditions.checkArgument( Preconditions.checkArgument(
renameReservedMap != null && renameReservedMap != null &&
@ -1132,7 +1149,7 @@ private static byte[] renameReservedComponentOnUpgrade(byte[] component,
private static byte[] renameReservedRootComponentOnUpgrade(byte[] component, private static byte[] renameReservedRootComponentOnUpgrade(byte[] component,
final int layoutVersion) { final int layoutVersion) {
// If the LV doesn't support inode IDs, we're doing an upgrade // If the LV doesn't support inode IDs, we're doing an upgrade
if (!LayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) { if (!NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) {
if (Arrays.equals(component, FSDirectory.DOT_RESERVED)) { if (Arrays.equals(component, FSDirectory.DOT_RESERVED)) {
Preconditions.checkArgument( Preconditions.checkArgument(
renameReservedMap != null && renameReservedMap != null &&
@ -1215,7 +1232,7 @@ void save(File newFile, FSImageCompression compression) throws IOException {
DigestOutputStream fos = new DigestOutputStream(fout, digester); DigestOutputStream fos = new DigestOutputStream(fout, digester);
DataOutputStream out = new DataOutputStream(fos); DataOutputStream out = new DataOutputStream(fos);
try { try {
out.writeInt(HdfsConstants.LAYOUT_VERSION); out.writeInt(HdfsConstants.NAMENODE_LAYOUT_VERSION);
LayoutFlags.write(out); LayoutFlags.write(out);
// We use the non-locked version of getNamespaceInfo here since // We use the non-locked version of getNamespaceInfo here since
// the coordinating thread of saveNamespace already has read-locked // the coordinating thread of saveNamespace already has read-locked

View File

@ -42,7 +42,6 @@
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.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
@ -281,6 +280,12 @@ private void loadNameSystemSection(InputStream in) throws IOException {
fsn.setGenerationStampV1Limit(s.getGenstampV1Limit()); fsn.setGenerationStampV1Limit(s.getGenstampV1Limit());
fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId()); fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
imgTxId = s.getTransactionId(); imgTxId = s.getTransactionId();
if (s.hasRollingUpgradeStartTime()
&& fsn.getFSImage().hasRollbackFSImage()) {
// we set the rollingUpgradeInfo only when we make sure we have the
// rollback image
fsn.setRollingUpgradeInfo(true, s.getRollingUpgradeStartTime());
}
} }
private void loadStringTableSection(InputStream in) throws IOException { private void loadStringTableSection(InputStream in) throws IOException {
@ -430,7 +435,7 @@ private void saveInternal(FileOutputStream fout,
FileSummary.Builder b = FileSummary.newBuilder() FileSummary.Builder b = FileSummary.newBuilder()
.setOndiskVersion(FSImageUtil.FILE_VERSION) .setOndiskVersion(FSImageUtil.FILE_VERSION)
.setLayoutVersion(LayoutVersion.getCurrentLayoutVersion()); .setLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
codec = compression.getImageCodec(); codec = compression.getImageCodec();
if (codec != null) { if (codec != null) {
@ -521,6 +526,9 @@ private void saveNameSystemSection(FileSummary.Builder summary)
// from the actual saver thread, there's a potential of a // from the actual saver thread, there's a potential of a
// fairness-related deadlock. See the comments on HDFS-2223. // fairness-related deadlock. See the comments on HDFS-2223.
b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID()); b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID());
if (fsn.isRollingUpgrade()) {
b.setRollingUpgradeStartTime(fsn.getRollingUpgradeInfo().getStartTime());
}
NameSystemSection s = b.build(); NameSystemSection s = b.build();
s.writeDelimitedTo(out); s.writeDelimitedTo(out);

View File

@ -33,7 +33,6 @@
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@ -118,8 +117,9 @@ static INodeFile readINodeUnderConstruction(
DataInput in, FSNamesystem fsNamesys, int imgVersion) DataInput in, FSNamesystem fsNamesys, int imgVersion)
throws IOException { throws IOException {
byte[] name = readBytes(in); byte[] name = readBytes(in);
long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in long inodeId = NameNodeLayoutVersion.supports(
.readLong() : fsNamesys.allocateNewInodeId(); LayoutVersion.Feature.ADD_INODE_ID, imgVersion) ? in.readLong()
: fsNamesys.allocateNewInodeId();
short blockReplication = in.readShort(); short blockReplication = in.readShort();
long modificationTime = in.readLong(); long modificationTime = in.readLong();
long preferredBlockSize = in.readLong(); long preferredBlockSize = in.readLong();

View File

@ -22,6 +22,7 @@
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.EnumSet;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -40,7 +41,7 @@
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -54,8 +55,28 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
List<FSImageFile> foundImages = new ArrayList<FSImageFile>(); List<FSImageFile> foundImages = new ArrayList<FSImageFile>();
private long maxSeenTxId = 0; private long maxSeenTxId = 0;
private static final Pattern IMAGE_REGEX = Pattern.compile( private final List<Pattern> namePatterns = Lists.newArrayList();
NameNodeFile.IMAGE.getName() + "_(\\d+)");
FSImageTransactionalStorageInspector() {
this(EnumSet.of(NameNodeFile.IMAGE));
}
FSImageTransactionalStorageInspector(EnumSet<NameNodeFile> nnfs) {
for (NameNodeFile nnf : nnfs) {
Pattern pattern = Pattern.compile(nnf.getName() + "_(\\d+)");
namePatterns.add(pattern);
}
}
private Matcher matchPattern(String name) {
for (Pattern p : namePatterns) {
Matcher m = p.matcher(name);
if (m.matches()) {
return m;
}
}
return null;
}
@Override @Override
public void inspectDirectory(StorageDirectory sd) throws IOException { public void inspectDirectory(StorageDirectory sd) throws IOException {
@ -90,8 +111,8 @@ public void inspectDirectory(StorageDirectory sd) throws IOException {
String name = f.getName(); String name = f.getName();
// Check for fsimage_* // Check for fsimage_*
Matcher imageMatch = IMAGE_REGEX.matcher(name); Matcher imageMatch = this.matchPattern(name);
if (imageMatch.matches()) { if (imageMatch != null) {
if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) { if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
try { try {
long txid = Long.valueOf(imageMatch.group(1)); long txid = Long.valueOf(imageMatch.group(1));

View File

@ -25,7 +25,6 @@
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;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.Loader; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.Loader;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
@ -71,7 +70,7 @@ public static FileSummary loadSummary(RandomAccessFile file)
+ summary.getOndiskVersion()); + summary.getOndiskVersion());
} }
if (!LayoutVersion.supports(Feature.PROTOBUF_FORMAT, if (!NameNodeLayoutVersion.supports(Feature.PROTOBUF_FORMAT,
summary.getLayoutVersion())) { summary.getLayoutVersion())) {
throw new IOException("Unsupported layout version " throw new IOException("Unsupported layout version "
+ summary.getLayoutVersion()); + summary.getLayoutVersion());

View File

@ -36,10 +36,10 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
@ -175,6 +175,8 @@
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -196,20 +198,20 @@
import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Util; import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection.PersistToken;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream; import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease; import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer; import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext; import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer; import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean; import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@ -391,6 +393,14 @@ private void logAuditEvent(boolean succeeded,
private final CacheManager cacheManager; private final CacheManager cacheManager;
private final DatanodeStatistics datanodeStatistics; private final DatanodeStatistics datanodeStatistics;
private RollingUpgradeInfo rollingUpgradeInfo = null;
/**
* A flag that indicates whether the checkpointer should checkpoint a rollback
* fsimage. The edit log tailer sets this flag. The checkpoint will create a
* rollback fsimage if the flag is true, and then change the flag to false.
*/
private volatile boolean needRollbackFsImage;
// Block pool ID used by this namenode // Block pool ID used by this namenode
private String blockPoolId; private String blockPoolId;
@ -484,7 +494,10 @@ private void logAuditEvent(boolean succeeded,
private HAContext haContext; private HAContext haContext;
private final boolean haEnabled; private final boolean haEnabled;
/** flag indicating whether replication queues have been initialized */
boolean initializedReplQueues = false;
/** /**
* Whether the namenode is in the middle of starting the active service * Whether the namenode is in the middle of starting the active service
*/ */
@ -605,8 +618,7 @@ private static void checkConfiguration(Configuration conf)
* @return an FSNamesystem which contains the loaded namespace * @return an FSNamesystem which contains the loaded namespace
* @throws IOException if loading fails * @throws IOException if loading fails
*/ */
public static FSNamesystem loadFromDisk(Configuration conf) static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
throws IOException {
checkConfiguration(conf); checkConfiguration(conf);
FSImage fsImage = new FSImage(conf, FSImage fsImage = new FSImage(conf,
@ -619,10 +631,8 @@ public static FSNamesystem loadFromDisk(Configuration conf)
} }
long loadStart = now(); long loadStart = now();
String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
try { try {
namesystem.loadFSImage(startOpt, fsImage, namesystem.loadFSImage(startOpt);
HAUtil.isHAEnabled(conf, nameserviceId));
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.warn("Encountered exception loading fsimage", ioe); LOG.warn("Encountered exception loading fsimage", ioe);
fsImage.close(); fsImage.close();
@ -848,8 +858,9 @@ private List<AuditLogger> initAuditLoggers(Configuration conf) {
return Collections.unmodifiableList(auditLoggers); return Collections.unmodifiableList(auditLoggers);
} }
void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled) private void loadFSImage(StartupOption startOpt) throws IOException {
throws IOException { final FSImage fsImage = getFSImage();
// format before starting up if requested // format before starting up if requested
if (startOpt == StartupOption.FORMAT) { if (startOpt == StartupOption.FORMAT) {
@ -862,8 +873,15 @@ void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled)
try { try {
// We shouldn't be calling saveNamespace if we've come up in standby state. // We shouldn't be calling saveNamespace if we've come up in standby state.
MetaRecoveryContext recovery = startOpt.createRecoveryContext(); MetaRecoveryContext recovery = startOpt.createRecoveryContext();
boolean needToSave = final boolean staleImage
fsImage.recoverTransitionRead(startOpt, this, recovery) && !haEnabled; = fsImage.recoverTransitionRead(startOpt, this, recovery);
if (RollingUpgradeStartupOption.ROLLBACK.matches(startOpt)) {
rollingUpgradeInfo = null;
}
final boolean needToSave = staleImage && !haEnabled && !isRollingUpgrade();
LOG.info("Need to save fs image? " + needToSave
+ " (staleImage=" + staleImage + ", haEnabled=" + haEnabled
+ ", isRollingUpgrade=" + isRollingUpgrade() + ")");
if (needToSave) { if (needToSave) {
fsImage.saveNamespace(this); fsImage.saveNamespace(this);
} else { } else {
@ -926,8 +944,7 @@ void startCommonServices(Configuration conf, HAContext haContext) throws IOExcep
try { try {
nnResourceChecker = new NameNodeResourceChecker(conf); nnResourceChecker = new NameNodeResourceChecker(conf);
checkAvailableResources(); checkAvailableResources();
assert safeMode != null && assert safeMode != null && !isPopulatingReplQueues();
!safeMode.isPopulatingReplQueues();
StartupProgress prog = NameNode.getStartupProgress(); StartupProgress prog = NameNode.getStartupProgress();
prog.beginPhase(Phase.SAFEMODE); prog.beginPhase(Phase.SAFEMODE);
prog.setTotal(Phase.SAFEMODE, STEP_AWAITING_REPORTED_BLOCKS, prog.setTotal(Phase.SAFEMODE, STEP_AWAITING_REPORTED_BLOCKS,
@ -982,12 +999,12 @@ void startActiveServices() throws IOException {
blockManager.clearQueues(); blockManager.clearQueues();
blockManager.processAllPendingDNMessages(); blockManager.processAllPendingDNMessages();
if (!isInSafeMode() || // Only need to re-process the queue, If not in SafeMode.
(isInSafeMode() && safeMode.isPopulatingReplQueues())) { if (!isInSafeMode()) {
LOG.info("Reprocessing replication and invalidation queues"); LOG.info("Reprocessing replication and invalidation queues");
blockManager.processMisReplicatedBlocks(); initializeReplQueues();
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NameNode metadata after re-processing " + LOG.debug("NameNode metadata after re-processing " +
"replication and invalidation queues during failover:\n" + "replication and invalidation queues during failover:\n" +
@ -1026,7 +1043,16 @@ void startActiveServices() throws IOException {
startingActiveService = false; startingActiveService = false;
} }
} }
/**
* Initialize replication queues.
*/
private void initializeReplQueues() {
LOG.info("initializing replication queues");
blockManager.processMisReplicatedBlocks();
initializedReplQueues = true;
}
/** /**
* @return Whether the namenode is transitioning to active state and is in the * @return Whether the namenode is transitioning to active state and is in the
* middle of the {@link #startActiveServices()} * middle of the {@link #startActiveServices()}
@ -1074,6 +1100,9 @@ void stopActiveServices() {
cacheManager.clearDirectiveStats(); cacheManager.clearDirectiveStats();
blockManager.getDatanodeManager().clearPendingCachingCommands(); blockManager.getDatanodeManager().clearPendingCachingCommands();
blockManager.getDatanodeManager().setShouldSendCachingCommands(false); blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
// Don't want to keep replication queues when not in Active.
blockManager.clearQueues();
initializedReplQueues = false;
} finally { } finally {
writeUnlock(); writeUnlock();
} }
@ -1101,6 +1130,16 @@ void startStandbyServices(final Configuration conf) throws IOException {
} }
} }
/**
* Called when the NN is in Standby state and the editlog tailer tails the
* OP_ROLLING_UPGRADE_START.
*/
void triggerRollbackCheckpoint() {
setNeedRollbackFsImage(true);
if (standbyCheckpointer != null) {
standbyCheckpointer.triggerRollbackCheckpoint();
}
}
/** /**
* Called while the NN is in Standby state, but just about to be * Called while the NN is in Standby state, but just about to be
@ -4201,23 +4240,24 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
throws IOException { throws IOException {
readLock(); readLock();
try { try {
//get datanode commands
final int maxTransfer = blockManager.getMaxReplicationStreams() final int maxTransfer = blockManager.getMaxReplicationStreams()
- xmitsInProgress; - xmitsInProgress;
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed, nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
xceiverCount, maxTransfer, failedVolumes); xceiverCount, maxTransfer, failedVolumes);
return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
//create ha status
final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
haContext.getState().getServiceState(),
getFSImage().getLastAppliedOrWrittenTxId());
return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo);
} finally { } finally {
readUnlock(); readUnlock();
} }
} }
private NNHAStatusHeartbeat createHaStatusHeartbeat() {
HAState state = haContext.getState();
return new NNHAStatusHeartbeat(state.getServiceState(),
getFSImage().getLastAppliedOrWrittenTxId());
}
/** /**
* Returns whether or not there were available resources at the last check of * Returns whether or not there were available resources at the last check of
* resources. * resources.
@ -4501,6 +4541,7 @@ void saveNamespace() throws AccessControlException, IOException {
readLock(); readLock();
try { try {
checkOperation(OperationCategory.UNCHECKED); checkOperation(OperationCategory.UNCHECKED);
if (!isInSafeMode()) { if (!isInSafeMode()) {
throw new IOException("Safe mode should be turned ON " throw new IOException("Safe mode should be turned ON "
+ "in order to create namespace image."); + "in order to create namespace image.");
@ -4602,7 +4643,6 @@ public class SafeModeInfo {
private int safeReplication; private int safeReplication;
/** threshold for populating needed replication queues */ /** threshold for populating needed replication queues */
private double replQueueThreshold; private double replQueueThreshold;
// internal fields // internal fields
/** Time when threshold was reached. /** Time when threshold was reached.
* <br> -1 safe mode is off * <br> -1 safe mode is off
@ -4620,8 +4660,6 @@ public class SafeModeInfo {
private int blockReplQueueThreshold; private int blockReplQueueThreshold;
/** time of the last status printout */ /** time of the last status printout */
private long lastStatusReport = 0; private long lastStatusReport = 0;
/** flag indicating whether replication queues have been initialized */
boolean initializedReplQueues = false;
/** Was safemode entered automatically because available resources were low. */ /** Was safemode entered automatically because available resources were low. */
private boolean resourcesLow = false; private boolean resourcesLow = false;
/** Should safemode adjust its block totals as blocks come in */ /** Should safemode adjust its block totals as blocks come in */
@ -4681,7 +4719,7 @@ private boolean shouldIncrementallyTrackBlocks() {
* *
* @see SafeModeInfo * @see SafeModeInfo
*/ */
private SafeModeInfo(boolean resourcesLow, boolean isReplQueuesInited) { private SafeModeInfo(boolean resourcesLow) {
this.threshold = 1.5f; // this threshold can never be reached this.threshold = 1.5f; // this threshold can never be reached
this.datanodeThreshold = Integer.MAX_VALUE; this.datanodeThreshold = Integer.MAX_VALUE;
this.extension = Integer.MAX_VALUE; this.extension = Integer.MAX_VALUE;
@ -4690,7 +4728,6 @@ private SafeModeInfo(boolean resourcesLow, boolean isReplQueuesInited) {
this.blockTotal = -1; this.blockTotal = -1;
this.blockSafe = -1; this.blockSafe = -1;
this.resourcesLow = resourcesLow; this.resourcesLow = resourcesLow;
this.initializedReplQueues = isReplQueuesInited;
enter(); enter();
reportStatus("STATE* Safe mode is ON.", true); reportStatus("STATE* Safe mode is ON.", true);
} }
@ -4704,13 +4741,6 @@ private synchronized boolean isOn() {
return this.reached >= 0; return this.reached >= 0;
} }
/**
* Check if we are populating replication queues.
*/
private synchronized boolean isPopulatingReplQueues() {
return initializedReplQueues;
}
/** /**
* Enter safe mode. * Enter safe mode.
*/ */
@ -4757,21 +4787,6 @@ private synchronized void leave() {
} }
} }
/**
* Initialize replication queues.
*/
private synchronized void initializeReplQueues() {
LOG.info("initializing replication queues");
assert !isPopulatingReplQueues() : "Already initialized repl queues";
long startTimeMisReplicatedScan = now();
blockManager.processMisReplicatedBlocks();
initializedReplQueues = true;
NameNode.stateChangeLog.info("STATE* Replication Queue initialization "
+ "scan for invalid, over- and under-replicated blocks "
+ "completed in " + (now() - startTimeMisReplicatedScan)
+ " msec");
}
/** /**
* Check whether we have reached the threshold for * Check whether we have reached the threshold for
* initializing replication queues. * initializing replication queues.
@ -4819,7 +4834,8 @@ private void checkMode() {
if (smmthread == null && needEnter()) { if (smmthread == null && needEnter()) {
enter(); enter();
// check if we are ready to initialize replication queues // check if we are ready to initialize replication queues
if (canInitializeReplQueues() && !isPopulatingReplQueues()) { if (canInitializeReplQueues() && !isPopulatingReplQueues()
&& !haEnabled) {
initializeReplQueues(); initializeReplQueues();
} }
reportStatus("STATE* Safe mode ON.", false); reportStatus("STATE* Safe mode ON.", false);
@ -4844,7 +4860,7 @@ private void checkMode() {
} }
// check if we are ready to initialize replication queues // check if we are ready to initialize replication queues
if (canInitializeReplQueues() && !isPopulatingReplQueues()) { if (canInitializeReplQueues() && !isPopulatingReplQueues() && !haEnabled) {
initializeReplQueues(); initializeReplQueues();
} }
} }
@ -5154,11 +5170,7 @@ public boolean isPopulatingReplQueues() {
if (!shouldPopulateReplQueues()) { if (!shouldPopulateReplQueues()) {
return false; return false;
} }
// safeMode is volatile, and may be set to null at any time return initializedReplQueues;
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null)
return true;
return safeMode.isPopulatingReplQueues();
} }
private boolean shouldPopulateReplQueues() { private boolean shouldPopulateReplQueues() {
@ -5278,7 +5290,7 @@ void enterSafeMode(boolean resourcesLow) throws IOException {
getEditLog().logSyncAll(); getEditLog().logSyncAll();
} }
if (!isInSafeMode()) { if (!isInSafeMode()) {
safeMode = new SafeModeInfo(resourcesLow, isPopulatingReplQueues()); safeMode = new SafeModeInfo(resourcesLow);
return; return;
} }
if (resourcesLow) { if (resourcesLow) {
@ -5353,8 +5365,8 @@ NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
NamenodeCommand cmd = null; NamenodeCommand cmd = null;
try { try {
checkOperation(OperationCategory.CHECKPOINT); checkOperation(OperationCategory.CHECKPOINT);
checkNameNodeSafeMode("Checkpoint not started"); checkNameNodeSafeMode("Checkpoint not started");
LOG.info("Start checkpoint for " + backupNode.getAddress()); LOG.info("Start checkpoint for " + backupNode.getAddress());
cmd = getFSImage().startCheckpoint(backupNode, activeNamenode); cmd = getFSImage().startCheckpoint(backupNode, activeNamenode);
getEditLog().logSync(); getEditLog().logSync();
@ -7148,6 +7160,173 @@ void removeSnapshottableDirs(List<INodeDirectorySnapshottable> toRemove) {
} }
} }
RollingUpgradeInfo queryRollingUpgrade() throws IOException {
checkSuperuserPrivilege();
checkOperation(OperationCategory.READ);
readLock();
try {
if (rollingUpgradeInfo != null) {
boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
}
return rollingUpgradeInfo;
} finally {
readUnlock();
}
}
RollingUpgradeInfo startRollingUpgrade() throws IOException {
checkSuperuserPrivilege();
checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
long startTime = now();
if (!haEnabled) { // for non-HA, we require NN to be in safemode
startRollingUpgradeInternalForNonHA(startTime);
} else { // for HA, NN cannot be in safemode
checkNameNodeSafeMode("Failed to start rolling upgrade");
startRollingUpgradeInternal(startTime);
}
getEditLog().logStartRollingUpgrade(rollingUpgradeInfo.getStartTime());
if (haEnabled) {
// roll the edit log to make sure the standby NameNode can tail
getFSImage().rollEditLog();
}
} finally {
writeUnlock();
}
getEditLog().logSync();
if (auditLog.isInfoEnabled() && isExternalInvocation()) {
logAuditEvent(true, "startRollingUpgrade", null, null, null);
}
return rollingUpgradeInfo;
}
/**
* Update internal state to indicate that a rolling upgrade is in progress.
* @param startTime
*/
void startRollingUpgradeInternal(long startTime)
throws IOException {
checkRollingUpgrade("start rolling upgrade");
getFSImage().checkUpgrade(this);
setRollingUpgradeInfo(false, startTime);
}
/**
* Update internal state to indicate that a rolling upgrade is in progress for
* non-HA setup. This requires the namesystem is in SafeMode and after doing a
* checkpoint for rollback the namesystem will quit the safemode automatically
*/
private void startRollingUpgradeInternalForNonHA(long startTime)
throws IOException {
Preconditions.checkState(!haEnabled);
if (!isInSafeMode()) {
throw new IOException("Safe mode should be turned ON "
+ "in order to create namespace image.");
}
checkRollingUpgrade("start rolling upgrade");
getFSImage().checkUpgrade(this);
// in non-HA setup, we do an extra ckpt to generate a rollback image
getFSImage().saveNamespace(this, NameNodeFile.IMAGE_ROLLBACK, null);
LOG.info("Successfully saved namespace for preparing rolling upgrade.");
// leave SafeMode automatically
setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
setRollingUpgradeInfo(true, startTime);
}
void setRollingUpgradeInfo(boolean createdRollbackImages, long startTime) {
rollingUpgradeInfo = new RollingUpgradeInfo(blockPoolId,
createdRollbackImages, startTime, 0L);
}
public void setCreatedRollbackImages(boolean created) {
if (rollingUpgradeInfo != null) {
rollingUpgradeInfo.setCreatedRollbackImages(created);
}
}
public RollingUpgradeInfo getRollingUpgradeInfo() {
return rollingUpgradeInfo;
}
public boolean isNeedRollbackFsImage() {
return needRollbackFsImage;
}
public void setNeedRollbackFsImage(boolean needRollbackFsImage) {
this.needRollbackFsImage = needRollbackFsImage;
}
@Override // NameNodeMXBean
public RollingUpgradeInfo.Bean getRollingUpgradeStatus() {
readLock();
try {
RollingUpgradeInfo upgradeInfo = getRollingUpgradeInfo();
if (upgradeInfo != null) {
return new RollingUpgradeInfo.Bean(upgradeInfo);
}
return null;
} finally {
readUnlock();
}
}
/** Is rolling upgrade in progress? */
public boolean isRollingUpgrade() {
return rollingUpgradeInfo != null;
}
void checkRollingUpgrade(String action) throws RollingUpgradeException {
if (isRollingUpgrade()) {
throw new RollingUpgradeException("Failed to " + action
+ " since a rolling upgrade is already in progress."
+ " Existing rolling upgrade info:\n" + rollingUpgradeInfo);
}
}
RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
checkSuperuserPrivilege();
checkOperation(OperationCategory.WRITE);
writeLock();
final RollingUpgradeInfo returnInfo;
try {
checkOperation(OperationCategory.WRITE);
checkNameNodeSafeMode("Failed to finalize rolling upgrade");
returnInfo = finalizeRollingUpgradeInternal(now());
getEditLog().logFinalizeRollingUpgrade(returnInfo.getFinalizeTime());
getFSImage().saveNamespace(this);
getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK,
NameNodeFile.IMAGE);
} finally {
writeUnlock();
}
// getEditLog().logSync() is not needed since it does saveNamespace
if (auditLog.isInfoEnabled() && isExternalInvocation()) {
logAuditEvent(true, "finalizeRollingUpgrade", null, null, null);
}
return returnInfo;
}
RollingUpgradeInfo finalizeRollingUpgradeInternal(long finalizeTime)
throws RollingUpgradeException {
if (!isRollingUpgrade()) {
throw new RollingUpgradeException(
"Failed to finalize rolling upgrade since there is no rolling upgrade in progress.");
}
final long startTime = rollingUpgradeInfo.getStartTime();
rollingUpgradeInfo = null;
return new RollingUpgradeInfo(blockPoolId, false, startTime, finalizeTime);
}
long addCacheDirective(CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) long addCacheDirective(CacheDirectiveInfo directive, EnumSet<CacheFlag> flags)
throws IOException { throws IOException {
checkOperation(OperationCategory.WRITE); checkOperation(OperationCategory.WRITE);

View File

@ -196,6 +196,32 @@ public List<RemoteEditLog> getRemoteEditLogs(long firstTxId,
return ret; return ret;
} }
/**
* Discard all editlog segments whose first txid is greater than or equal to
* the given txid, by renaming them with suffix ".trash".
*/
private void discardEditLogSegments(long startTxId) throws IOException {
File currentDir = sd.getCurrentDir();
List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
List<EditLogFile> toTrash = Lists.newArrayList();
LOG.info("Discard the EditLog files, the given start txid is " + startTxId);
// go through the editlog files to make sure the startTxId is right at the
// segment boundary
for (EditLogFile elf : allLogFiles) {
if (elf.getFirstTxId() >= startTxId) {
toTrash.add(elf);
} else {
Preconditions.checkState(elf.getLastTxId() < startTxId);
}
}
for (EditLogFile elf : toTrash) {
// rename these editlog file as .trash
elf.moveAsideTrashFile(startTxId);
LOG.info("Trash the EditLog file " + elf);
}
}
/** /**
* returns matching edit logs via the log directory. Simple helper function * returns matching edit logs via the log directory. Simple helper function
@ -465,6 +491,11 @@ void moveAsideCorruptFile() throws IOException {
renameSelf(".corrupt"); renameSelf(".corrupt");
} }
void moveAsideTrashFile(long markerTxid) throws IOException {
assert this.getFirstTxId() >= markerTxid;
renameSelf(".trash");
}
public void moveAsideEmptyFile() throws IOException { public void moveAsideEmptyFile() throws IOException {
assert lastTxId == HdfsConstants.INVALID_TXID; assert lastTxId == HdfsConstants.INVALID_TXID;
renameSelf(".empty"); renameSelf(".empty");
@ -489,4 +520,9 @@ public String toString() {
isInProgress(), hasCorruptHeader); isInProgress(), hasCorruptHeader);
} }
} }
@Override
public void discardSegments(long startTxid) throws IOException {
discardEditLogSegments(startTxid);
}
} }

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@ -77,7 +78,8 @@ public class GetImageServlet extends HttpServlet {
private static final String END_TXID_PARAM = "endTxId"; private static final String END_TXID_PARAM = "endTxId";
private static final String STORAGEINFO_PARAM = "storageInfo"; private static final String STORAGEINFO_PARAM = "storageInfo";
private static final String LATEST_FSIMAGE_VALUE = "latest"; private static final String LATEST_FSIMAGE_VALUE = "latest";
private static final String IMAGE_FILE_TYPE = "imageFile";
private static Set<Long> currentlyDownloadingCheckpoints = private static Set<Long> currentlyDownloadingCheckpoints =
Collections.<Long>synchronizedSet(new HashSet<Long>()); Collections.<Long>synchronizedSet(new HashSet<Long>());
@ -86,7 +88,7 @@ public void doGet(final HttpServletRequest request,
final HttpServletResponse response final HttpServletResponse response
) throws ServletException, IOException { ) throws ServletException, IOException {
try { try {
ServletContext context = getServletContext(); final ServletContext context = getServletContext();
final FSImage nnImage = NameNodeHttpServer.getFsImageFromContext(context); final FSImage nnImage = NameNodeHttpServer.getFsImageFromContext(context);
final GetImageParams parsedParams = new GetImageParams(request, response); final GetImageParams parsedParams = new GetImageParams(request, response);
final Configuration conf = (Configuration) context final Configuration conf = (Configuration) context
@ -126,7 +128,8 @@ public Void run() throws Exception {
imageFile = nnImage.getStorage().getHighestFsImageName(); imageFile = nnImage.getStorage().getHighestFsImageName();
} else { } else {
errorMessage += " with txid " + txid; errorMessage += " with txid " + txid;
imageFile = nnImage.getStorage().getFsImageName(txid); imageFile = nnImage.getStorage().getFsImage(txid,
EnumSet.of(NameNodeFile.IMAGE, NameNodeFile.IMAGE_ROLLBACK));
} }
if (imageFile == null) { if (imageFile == null) {
throw new IOException(errorMessage); throw new IOException(errorMessage);
@ -154,6 +157,7 @@ public Void run() throws Exception {
} }
} else if (parsedParams.isPutImage()) { } else if (parsedParams.isPutImage()) {
final long txid = parsedParams.getTxId(); final long txid = parsedParams.getTxId();
final NameNodeFile nnf = parsedParams.getNameNodeFile();
if (! currentlyDownloadingCheckpoints.add(txid)) { if (! currentlyDownloadingCheckpoints.add(txid)) {
response.sendError(HttpServletResponse.SC_CONFLICT, response.sendError(HttpServletResponse.SC_CONFLICT,
@ -163,7 +167,7 @@ public Void run() throws Exception {
} }
try { try {
if (nnImage.getStorage().findImageFile(txid) != null) { if (nnImage.getStorage().findImageFile(nnf, txid) != null) {
response.sendError(HttpServletResponse.SC_CONFLICT, response.sendError(HttpServletResponse.SC_CONFLICT,
"Another checkpointer already uploaded an checkpoint " + "Another checkpointer already uploaded an checkpoint " +
"for txid " + txid); "for txid " + txid);
@ -177,11 +181,15 @@ public Void run() throws Exception {
long start = now(); long start = now();
// issue a HTTP get request to download the new fsimage // issue a HTTP get request to download the new fsimage
MD5Hash downloadImageDigest = MD5Hash downloadImageDigest = TransferFsImage
TransferFsImage.downloadImageToStorage( .downloadImageToStorage(parsedParams.getInfoServer(conf),
parsedParams.getInfoServer(conf), txid, txid, nnImage.getStorage(), true);
nnImage.getStorage(), true); nnImage.saveDigestAndRenameCheckpointImage(nnf, txid,
nnImage.saveDigestAndRenameCheckpointImage(txid, downloadImageDigest); downloadImageDigest);
if (nnf == NameNodeFile.IMAGE_ROLLBACK) {
NameNodeHttpServer.getNameNodeFromContext(context)
.getNamesystem().setCreatedRollbackImages(true);
}
if (metrics != null) { // Metrics non-null only when used inside name node if (metrics != null) { // Metrics non-null only when used inside name node
long elapsed = now() - start; long elapsed = now() - start;
@ -190,7 +198,7 @@ public Void run() throws Exception {
// Now that we have a new checkpoint, we might be able to // Now that we have a new checkpoint, we might be able to
// remove some old ones. // remove some old ones.
nnImage.purgeOldStorage(); nnImage.purgeOldStorage(nnf);
} finally { } finally {
currentlyDownloadingCheckpoints.remove(txid); currentlyDownloadingCheckpoints.remove(txid);
} }
@ -314,9 +322,12 @@ static String getParamStringForMostRecentImage() {
return "getimage=1&" + TXID_PARAM + "=" + LATEST_FSIMAGE_VALUE; return "getimage=1&" + TXID_PARAM + "=" + LATEST_FSIMAGE_VALUE;
} }
static String getParamStringForImage(long txid, static String getParamStringForImage(NameNodeFile nnf, long txid,
StorageInfo remoteStorageInfo) { StorageInfo remoteStorageInfo) {
final String imageType = nnf == null ? "" : "&" + IMAGE_FILE_TYPE + "="
+ nnf.name();
return "getimage=1&" + TXID_PARAM + "=" + txid return "getimage=1&" + TXID_PARAM + "=" + txid
+ imageType
+ "&" + STORAGEINFO_PARAM + "=" + + "&" + STORAGEINFO_PARAM + "=" +
remoteStorageInfo.toColonSeparatedString(); remoteStorageInfo.toColonSeparatedString();
} }
@ -329,7 +340,7 @@ static String getParamStringForLog(RemoteEditLog log,
remoteStorageInfo.toColonSeparatedString(); remoteStorageInfo.toColonSeparatedString();
} }
static String getParamStringToPutImage(long txid, static String getParamStringToPutImage(NameNodeFile nnf, long txid,
URL url, Storage storage) { URL url, Storage storage) {
InetSocketAddress imageListenAddress = NetUtils.createSocketAddr(url InetSocketAddress imageListenAddress = NetUtils.createSocketAddr(url
.getAuthority()); .getAuthority());
@ -338,6 +349,7 @@ static String getParamStringToPutImage(long txid,
: imageListenAddress.getHostName(); : imageListenAddress.getHostName();
return "putimage=1" + return "putimage=1" +
"&" + TXID_PARAM + "=" + txid + "&" + TXID_PARAM + "=" + txid +
"&" + IMAGE_FILE_TYPE + "=" + nnf.name() +
"&port=" + imageListenAddress.getPort() + "&port=" + imageListenAddress.getPort() +
(machine != null ? "&machine=" + machine : "") (machine != null ? "&machine=" + machine : "")
+ "&" + STORAGEINFO_PARAM + "=" + + "&" + STORAGEINFO_PARAM + "=" +
@ -351,6 +363,7 @@ static class GetImageParams {
private boolean isPutImage; private boolean isPutImage;
private int remoteport; private int remoteport;
private String machineName; private String machineName;
private NameNodeFile nnf;
private long startTxId, endTxId, txId; private long startTxId, endTxId, txId;
private String storageInfoString; private String storageInfoString;
private boolean fetchLatest; private boolean fetchLatest;
@ -375,6 +388,9 @@ public GetImageParams(HttpServletRequest request,
isGetImage = true; isGetImage = true;
try { try {
txId = ServletUtil.parseLongParam(request, TXID_PARAM); txId = ServletUtil.parseLongParam(request, TXID_PARAM);
String imageType = ServletUtil.getParameter(request, IMAGE_FILE_TYPE);
nnf = imageType == null ? NameNodeFile.IMAGE : NameNodeFile
.valueOf(imageType);
} catch (NumberFormatException nfe) { } catch (NumberFormatException nfe) {
if (request.getParameter(TXID_PARAM).equals(LATEST_FSIMAGE_VALUE)) { if (request.getParameter(TXID_PARAM).equals(LATEST_FSIMAGE_VALUE)) {
fetchLatest = true; fetchLatest = true;
@ -389,6 +405,9 @@ public GetImageParams(HttpServletRequest request,
} else if (key.equals("putimage")) { } else if (key.equals("putimage")) {
isPutImage = true; isPutImage = true;
txId = ServletUtil.parseLongParam(request, TXID_PARAM); txId = ServletUtil.parseLongParam(request, TXID_PARAM);
String imageType = ServletUtil.getParameter(request, IMAGE_FILE_TYPE);
nnf = imageType == null ? NameNodeFile.IMAGE : NameNodeFile
.valueOf(imageType);
} else if (key.equals("port")) { } else if (key.equals("port")) {
remoteport = new Integer(val[0]).intValue(); remoteport = new Integer(val[0]).intValue();
} else if (key.equals("machine")) { } else if (key.equals("machine")) {
@ -419,7 +438,12 @@ public long getTxId() {
Preconditions.checkState(isGetImage || isPutImage); Preconditions.checkState(isGetImage || isPutImage);
return txId; return txId;
} }
public NameNodeFile getNameNodeFile() {
Preconditions.checkState(isPutImage || isGetImage);
return nnf;
}
public long getStartTxId() { public long getStartTxId() {
Preconditions.checkState(isGetEdit); Preconditions.checkState(isGetEdit);
return startTxId; return startTxId;

View File

@ -65,6 +65,14 @@ public interface JournalManager extends Closeable, LogsPurgeable,
*/ */
void recoverUnfinalizedSegments() throws IOException; void recoverUnfinalizedSegments() throws IOException;
/**
* Discard the segments whose first txid is >= the given txid.
* @param startTxId The given txid should be right at the segment boundary,
* i.e., it should be the first txid of some segment, if segment corresponding
* to the txid exists.
*/
void discardSegments(long startTxId) throws IOException;
/** /**
* Close the journal manager, freeing any resources it may hold. * Close the journal manager, freeing any resources it may hold.
*/ */

View File

@ -656,4 +656,10 @@ String getSyncTimes() {
} }
return buf.toString(); return buf.toString();
} }
@Override
public void discardSegments(long startTxId) throws IOException {
// This operation is handled by FSEditLog directly.
throw new UnsupportedOperationException();
}
} }

View File

@ -25,6 +25,7 @@
import java.net.UnknownHostException; import java.net.UnknownHostException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -38,7 +39,6 @@
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException; import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@ -47,13 +47,12 @@
import org.apache.hadoop.hdfs.server.common.Util; import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.util.PersistentLongFile; import org.apache.hadoop.hdfs.util.PersistentLongFile;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.DNS; import org.apache.hadoop.net.DNS;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import com.google.common.base.Preconditions;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
/** /**
@ -69,19 +68,21 @@ public class NNStorage extends Storage implements Closeable,
// //
// The filenames used for storing the images // The filenames used for storing the images
// //
enum NameNodeFile { public enum NameNodeFile {
IMAGE ("fsimage"), IMAGE ("fsimage"),
TIME ("fstime"), // from "old" pre-HDFS-1073 format TIME ("fstime"), // from "old" pre-HDFS-1073 format
SEEN_TXID ("seen_txid"), SEEN_TXID ("seen_txid"),
EDITS ("edits"), EDITS ("edits"),
IMAGE_NEW ("fsimage.ckpt"), IMAGE_NEW ("fsimage.ckpt"),
IMAGE_ROLLBACK("fsimage_rollback"),
EDITS_NEW ("edits.new"), // from "old" pre-HDFS-1073 format EDITS_NEW ("edits.new"), // from "old" pre-HDFS-1073 format
EDITS_INPROGRESS ("edits_inprogress"), EDITS_INPROGRESS ("edits_inprogress"),
EDITS_TMP ("edits_tmp"); EDITS_TMP ("edits_tmp");
private String fileName = null; private String fileName = null;
private NameNodeFile(String name) { this.fileName = name; } private NameNodeFile(String name) { this.fileName = name; }
String getName() { return fileName; } @VisibleForTesting
public String getName() { return fileName; }
} }
/** /**
@ -90,7 +91,8 @@ enum NameNodeFile {
* or of type EDITS which stores edits or of type IMAGE_AND_EDITS which * or of type EDITS which stores edits or of type IMAGE_AND_EDITS which
* stores both fsimage and edits. * stores both fsimage and edits.
*/ */
static enum NameNodeDirType implements StorageDirType { @VisibleForTesting
public static enum NameNodeDirType implements StorageDirType {
UNDEFINED, UNDEFINED,
IMAGE, IMAGE,
EDITS, EDITS,
@ -498,21 +500,42 @@ public File[] getFsImageNameCheckpoint(long txid) {
} }
/** /**
* Return the name of the image file. * @return The first image file with the given txid and image type.
* @return The name of the first image file.
*/ */
public File getFsImageName(long txid) { public File getFsImageName(long txid, NameNodeFile nnf) {
StorageDirectory sd = null; for (Iterator<StorageDirectory> it = dirIterator(NameNodeDirType.IMAGE);
for (Iterator<StorageDirectory> it = it.hasNext();) {
dirIterator(NameNodeDirType.IMAGE); it.hasNext();) { StorageDirectory sd = it.next();
sd = it.next(); File fsImage = getStorageFile(sd, nnf, txid);
File fsImage = getStorageFile(sd, NameNodeFile.IMAGE, txid); if (FileUtil.canRead(sd.getRoot()) && fsImage.exists()) {
if(FileUtil.canRead(sd.getRoot()) && fsImage.exists())
return fsImage; return fsImage;
}
} }
return null; return null;
} }
/**
* @return The first image file whose txid is the same with the given txid and
* image type is one of the given types.
*/
public File getFsImage(long txid, EnumSet<NameNodeFile> nnfs) {
for (Iterator<StorageDirectory> it = dirIterator(NameNodeDirType.IMAGE);
it.hasNext();) {
StorageDirectory sd = it.next();
for (NameNodeFile nnf : nnfs) {
File fsImage = getStorageFile(sd, nnf, txid);
if (FileUtil.canRead(sd.getRoot()) && fsImage.exists()) {
return fsImage;
}
}
}
return null;
}
public File getFsImageName(long txid) {
return getFsImageName(txid, NameNodeFile.IMAGE);
}
public File getHighestFsImageName() { public File getHighestFsImageName() {
return getFsImageName(getMostRecentCheckpointTxId()); return getFsImageName(getMostRecentCheckpointTxId());
} }
@ -533,7 +556,7 @@ private void format(StorageDirectory sd) throws IOException {
*/ */
public void format(NamespaceInfo nsInfo) throws IOException { public void format(NamespaceInfo nsInfo) throws IOException {
Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 || Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 ||
nsInfo.getLayoutVersion() == HdfsConstants.LAYOUT_VERSION, nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION,
"Bad layout version: %s", nsInfo.getLayoutVersion()); "Bad layout version: %s", nsInfo.getLayoutVersion());
this.setStorageInfo(nsInfo); this.setStorageInfo(nsInfo);
@ -552,7 +575,7 @@ public static NamespaceInfo newNamespaceInfo()
} }
public void format() throws IOException { public void format() throws IOException {
this.layoutVersion = HdfsConstants.LAYOUT_VERSION; this.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
for (Iterator<StorageDirectory> it = for (Iterator<StorageDirectory> it =
dirIterator(); it.hasNext();) { dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next(); StorageDirectory sd = it.next();
@ -589,7 +612,8 @@ protected void setFieldsFromProperties(
} }
// Set Block pool ID in version with federation support // Set Block pool ID in version with federation support
if (versionSupportsFederation()) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, getLayoutVersion())) {
String sbpid = props.getProperty("blockpoolID"); String sbpid = props.getProperty("blockpoolID");
setBlockPoolID(sd.getRoot(), sbpid); setBlockPoolID(sd.getRoot(), sbpid);
} }
@ -614,7 +638,7 @@ private void setDeprecatedPropertiesForUpgrade(Properties props) {
* This should only be used during upgrades. * This should only be used during upgrades.
*/ */
String getDeprecatedProperty(String prop) { String getDeprecatedProperty(String prop) {
assert getLayoutVersion() > HdfsConstants.LAYOUT_VERSION : assert getLayoutVersion() > HdfsConstants.NAMENODE_LAYOUT_VERSION :
"getDeprecatedProperty should only be done when loading " + "getDeprecatedProperty should only be done when loading " +
"storage from past versions during upgrade."; "storage from past versions during upgrade.";
return deprecatedProperties.get(prop); return deprecatedProperties.get(prop);
@ -636,7 +660,8 @@ protected void setPropertiesFromFields(Properties props,
) throws IOException { ) throws IOException {
super.setPropertiesFromFields(props, sd); super.setPropertiesFromFields(props, sd);
// Set blockpoolID in version with federation support // Set blockpoolID in version with federation support
if (versionSupportsFederation()) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, getLayoutVersion())) {
props.setProperty("blockpoolID", blockpoolID); props.setProperty("blockpoolID", blockpoolID);
} }
} }
@ -656,20 +681,26 @@ static File getStorageFile(StorageDirectory sd, NameNodeFile type) {
@VisibleForTesting @VisibleForTesting
public static String getCheckpointImageFileName(long txid) { public static String getCheckpointImageFileName(long txid) {
return String.format("%s_%019d", return getNameNodeFileName(NameNodeFile.IMAGE_NEW, txid);
NameNodeFile.IMAGE_NEW.getName(), txid);
} }
@VisibleForTesting @VisibleForTesting
public static String getImageFileName(long txid) { public static String getImageFileName(long txid) {
return String.format("%s_%019d", return getNameNodeFileName(NameNodeFile.IMAGE, txid);
NameNodeFile.IMAGE.getName(), txid);
} }
@VisibleForTesting
public static String getRollbackImageFileName(long txid) {
return getNameNodeFileName(NameNodeFile.IMAGE_ROLLBACK, txid);
}
private static String getNameNodeFileName(NameNodeFile nnf, long txid) {
return String.format("%s_%019d", nnf.getName(), txid);
}
@VisibleForTesting @VisibleForTesting
public static String getInProgressEditsFileName(long startTxId) { public static String getInProgressEditsFileName(long startTxId) {
return String.format("%s_%019d", NameNodeFile.EDITS_INPROGRESS.getName(), return getNameNodeFileName(NameNodeFile.EDITS_INPROGRESS, startTxId);
startTxId);
} }
static File getInProgressEditsFile(StorageDirectory sd, long startTxId) { static File getInProgressEditsFile(StorageDirectory sd, long startTxId) {
@ -687,12 +718,11 @@ static File getTemporaryEditsFile(StorageDirectory sd,
return new File(sd.getCurrentDir(), return new File(sd.getCurrentDir(),
getTemporaryEditsFileName(startTxId, endTxId, timestamp)); getTemporaryEditsFileName(startTxId, endTxId, timestamp));
} }
static File getImageFile(StorageDirectory sd, long txid) { static File getImageFile(StorageDirectory sd, NameNodeFile nnf, long txid) {
return new File(sd.getCurrentDir(), return new File(sd.getCurrentDir(), getNameNodeFileName(nnf, txid));
getImageFileName(txid));
} }
@VisibleForTesting @VisibleForTesting
public static String getFinalizedEditsFileName(long startTxId, long endTxId) { public static String getFinalizedEditsFileName(long startTxId, long endTxId) {
return String.format("%s_%019d-%019d", NameNodeFile.EDITS.getName(), return String.format("%s_%019d-%019d", NameNodeFile.EDITS.getName(),
@ -720,12 +750,12 @@ File findFinalizedEditsFile(long startTxId, long endTxId)
} }
/** /**
* Return the first readable image file for the given txid, or null * Return the first readable image file for the given txid and image type, or
* if no such image can be found * null if no such image can be found
*/ */
File findImageFile(long txid) { File findImageFile(NameNodeFile nnf, long txid) {
return findFile(NameNodeDirType.IMAGE, return findFile(NameNodeDirType.IMAGE,
getImageFileName(txid)); getNameNodeFileName(nnf, txid));
} }
/** /**
@ -808,7 +838,8 @@ void processStartupOptionsForUpgrade(StartupOption startOpt, int layoutVersion)
// If upgrade from a release that does not support federation, // If upgrade from a release that does not support federation,
// if clusterId is provided in the startupOptions use it. // if clusterId is provided in the startupOptions use it.
// Else generate a new cluster ID // Else generate a new cluster ID
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) { if (!NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
if (startOpt.getClusterId() == null) { if (startOpt.getClusterId() == null) {
startOpt.setClusterId(newClusterID()); startOpt.setClusterId(newClusterID());
} }
@ -969,7 +1000,7 @@ void inspectStorageDirs(FSImageStorageInspector inspector)
* <b>Note:</b> this can mutate the storage info fields (ctime, version, etc). * <b>Note:</b> this can mutate the storage info fields (ctime, version, etc).
* @throws IOException if no valid storage dirs are found or no valid layout version * @throws IOException if no valid storage dirs are found or no valid layout version
*/ */
FSImageStorageInspector readAndInspectDirs() FSImageStorageInspector readAndInspectDirs(EnumSet<NameNodeFile> fileTypes)
throws IOException { throws IOException {
Integer layoutVersion = null; Integer layoutVersion = null;
boolean multipleLV = false; boolean multipleLV = false;
@ -1005,8 +1036,9 @@ FSImageStorageInspector readAndInspectDirs()
// (ie edits_<txnid>) then use the new inspector, which will ignore // (ie edits_<txnid>) then use the new inspector, which will ignore
// the old format dirs. // the old format dirs.
FSImageStorageInspector inspector; FSImageStorageInspector inspector;
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, getLayoutVersion())) { if (NameNodeLayoutVersion.supports(
inspector = new FSImageTransactionalStorageInspector(); LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
inspector = new FSImageTransactionalStorageInspector(fileTypes);
} else { } else {
inspector = new FSImagePreTransactionalStorageInspector(); inspector = new FSImagePreTransactionalStorageInspector();
} }

View File

@ -22,6 +22,7 @@
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.EnumSet;
import java.util.List; import java.util.List;
import java.util.TreeSet; import java.util.TreeSet;
@ -31,6 +32,7 @@
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile; import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile; import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.hdfs.util.MD5FileUtils;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -88,13 +90,35 @@ public NNStorageRetentionManager(Configuration conf, NNStorage storage,
this(conf, storage, purgeableLogs, new DeletionStoragePurger()); this(conf, storage, purgeableLogs, new DeletionStoragePurger());
} }
public void purgeOldStorage() throws IOException { void purgeCheckpoints(NameNodeFile nnf) throws IOException {
purgeCheckpoinsAfter(nnf, -1);
}
void purgeCheckpoinsAfter(NameNodeFile nnf, long fromTxId)
throws IOException {
FSImageTransactionalStorageInspector inspector = FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector(); new FSImageTransactionalStorageInspector(EnumSet.of(nnf));
storage.inspectStorageDirs(inspector);
for (FSImageFile image : inspector.getFoundImages()) {
if (image.getCheckpointTxId() > fromTxId) {
purger.purgeImage(image);
}
}
}
void purgeOldStorage(NameNodeFile nnf) throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector(EnumSet.of(nnf));
storage.inspectStorageDirs(inspector); storage.inspectStorageDirs(inspector);
long minImageTxId = getImageTxIdToRetain(inspector); long minImageTxId = getImageTxIdToRetain(inspector);
purgeCheckpointsOlderThan(inspector, minImageTxId); purgeCheckpointsOlderThan(inspector, minImageTxId);
if (nnf == NameNodeFile.IMAGE_ROLLBACK) {
// do not purge edits for IMAGE_ROLLBACK.
return;
}
// If fsimage_N is the image we want to keep, then we need to keep // If fsimage_N is the image we want to keep, then we need to keep
// all txns > N. We can remove anything < N+1, since fsimage_N // all txns > N. We can remove anything < N+1, since fsimage_N
// reflects the state up to and including N. However, we also // reflects the state up to and including N. However, we also

View File

@ -21,6 +21,7 @@
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
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.HadoopIllegalArgumentException; import org.apache.hadoop.HadoopIllegalArgumentException;
@ -40,6 +41,7 @@
import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.ha.*; import org.apache.hadoop.hdfs.server.namenode.ha.*;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@ -64,6 +66,7 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import javax.management.ObjectName; import javax.management.ObjectName;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
@ -205,6 +208,9 @@ public static enum OperationCategory {
" [" + StartupOption.CLUSTERID.getName() + " cid]" + " [" + StartupOption.CLUSTERID.getName() + " cid]" +
" [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | [" " [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | ["
+ StartupOption.ROLLBACK.getName() + "] | [" + StartupOption.ROLLBACK.getName() + "] | ["
+ StartupOption.ROLLINGUPGRADE.getName() + " <"
+ RollingUpgradeStartupOption.DOWNGRADE.name().toLowerCase() + "|"
+ RollingUpgradeStartupOption.ROLLBACK.name().toLowerCase() + "> ] | ["
+ StartupOption.FINALIZE.getName() + "] | [" + StartupOption.FINALIZE.getName() + "] | ["
+ StartupOption.IMPORT.getName() + "] | [" + StartupOption.IMPORT.getName() + "] | ["
+ StartupOption.INITIALIZESHAREDEDITS.getName() + "] | [" + StartupOption.INITIALIZESHAREDEDITS.getName() + "] | ["
@ -1119,6 +1125,10 @@ static StartupOption parseArguments(String args[]) {
return null; return null;
} }
} }
} else if (StartupOption.ROLLINGUPGRADE.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.ROLLINGUPGRADE;
++i;
startOpt.setRollingUpgradeStartupOption(args[i]);
} else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) { } else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.ROLLBACK; startOpt = StartupOption.ROLLBACK;
} else if (StartupOption.FINALIZE.getName().equalsIgnoreCase(cmd)) { } else if (StartupOption.FINALIZE.getName().equalsIgnoreCase(cmd)) {
@ -1164,7 +1174,7 @@ static StartupOption parseArguments(String args[]) {
} }
private static void setStartupOption(Configuration conf, StartupOption opt) { private static void setStartupOption(Configuration conf, StartupOption opt) {
conf.set(DFS_NAMENODE_STARTUP_KEY, opt.toString()); conf.set(DFS_NAMENODE_STARTUP_KEY, opt.name());
} }
static StartupOption getStartupOption(Configuration conf) { static StartupOption getStartupOption(Configuration conf) {
@ -1194,7 +1204,7 @@ private static void doRecovery(StartupOption startOpt, Configuration conf)
FSNamesystem fsn = null; FSNamesystem fsn = null;
try { try {
fsn = FSNamesystem.loadFromDisk(conf); fsn = FSNamesystem.loadFromDisk(conf);
fsn.saveNamespace(); fsn.getFSImage().saveNamespace(fsn);
MetaRecoveryContext.LOG.info("RECOVERY COMPLETE"); MetaRecoveryContext.LOG.info("RECOVERY COMPLETE");
} catch (IOException e) { } catch (IOException e) {
MetaRecoveryContext.LOG.info("RECOVERY FAILED: caught exception", e); MetaRecoveryContext.LOG.info("RECOVERY FAILED: caught exception", e);
@ -1210,6 +1220,7 @@ private static void doRecovery(StartupOption startOpt, Configuration conf)
public static NameNode createNameNode(String argv[], Configuration conf) public static NameNode createNameNode(String argv[], Configuration conf)
throws IOException { throws IOException {
LOG.info("createNameNode " + Arrays.asList(argv));
if (conf == null) if (conf == null)
conf = new HdfsConfiguration(); conf = new HdfsConfiguration();
StartupOption startOpt = parseArguments(argv); StartupOption startOpt = parseArguments(argv);

View File

@ -0,0 +1,98 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.util.HashMap;
import java.util.Map;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.FeatureInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
@InterfaceAudience.Private
public class NameNodeLayoutVersion {
/** Build layout version and corresponding feature matrix */
public final static Map<Integer, SortedSet<LayoutFeature>> FEATURES
= new HashMap<Integer, SortedSet<LayoutFeature>>();
public static final int CURRENT_LAYOUT_VERSION
= LayoutVersion.getCurrentLayoutVersion(Feature.values());
static {
LayoutVersion.updateMap(FEATURES, LayoutVersion.Feature.values());
LayoutVersion.updateMap(FEATURES, NameNodeLayoutVersion.Feature.values());
}
public static SortedSet<LayoutFeature> getFeatures(int lv) {
return FEATURES.get(lv);
}
public static boolean supports(final LayoutFeature f, final int lv) {
return LayoutVersion.supports(FEATURES, f, lv);
}
/**
* Enums for features that change the layout version.
* <br><br>
* To add a new layout version:
* <ul>
* <li>Define a new enum constant with a short enum name, the new layout version
* and description of the added feature.</li>
* <li>When adding a layout version with an ancestor that is not same as
* its immediate predecessor, use the constructor where a specific ancestor
* can be passed.
* </li>
* </ul>
*/
public static enum Feature implements LayoutFeature {
ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false);
private final FeatureInfo info;
/**
* Feature that is added at layout version {@code lv} - 1.
* @param lv new layout version with the addition of this feature
* @param description description of the feature
*/
Feature(final int lv, final String description) {
this(lv, lv + 1, description, false);
}
/**
* NameNode feature that is added at layout version {@code ancestoryLV}.
* @param lv new layout version with the addition of this feature
* @param ancestorLV layout version from which the new lv is derived from.
* @param description description of the feature
* @param reserved true when this is a layout version reserved for previous
* versions
* @param features set of features that are to be enabled for this version
*/
Feature(final int lv, final int ancestorLV, final String description,
boolean reserved, Feature... features) {
info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
}
@Override
public FeatureInfo getInfo() {
return info;
}
}
}

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
/** /**
* This is the JMX management interface for namenode information * This is the JMX management interface for namenode information
@ -78,7 +79,14 @@ public interface NameNodeMXBean {
* @return true, if upgrade is finalized * @return true, if upgrade is finalized
*/ */
public boolean isUpgradeFinalized(); public boolean isUpgradeFinalized();
/**
* Gets the RollingUpgrade information
*
* @return Rolling upgrade information
*/
public RollingUpgradeInfo.Bean getRollingUpgradeStatus();
/** /**
* Gets total used space by data nodes for non DFS purposes such as storing * Gets total used space by data nodes for non DFS purposes such as storing
* temporary files on the local file system * temporary files on the local file system

View File

@ -37,6 +37,7 @@
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.CacheFlag; import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.ContentSummary;
@ -48,7 +49,6 @@
import org.apache.hadoop.fs.ParentNotDirectoryException; import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
@ -64,12 +64,11 @@
import org.apache.hadoop.hdfs.HDFSPolicyProvider; import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.protocol.AclException; import org.apache.hadoop.hdfs.protocol.AclException;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -78,6 +77,7 @@
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -85,6 +85,7 @@
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
@ -104,6 +105,7 @@
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods; import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@ -871,6 +873,21 @@ public void finalizeUpgrade() throws IOException {
namesystem.finalizeUpgrade(); namesystem.finalizeUpgrade();
} }
@Override // ClientProtocol
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
LOG.info("rollingUpgrade " + action);
switch(action) {
case QUERY:
return namesystem.queryRollingUpgrade();
case PREPARE:
return namesystem.startRollingUpgrade();
case FINALIZE:
return namesystem.finalizeRollingUpgrade();
default:
throw new UnsupportedActionException(action + " is not yet supported.");
}
}
@Override // ClientProtocol @Override // ClientProtocol
public void metaSave(String filename) throws IOException { public void metaSave(String filename) throws IOException {
namesystem.metaSave(filename); namesystem.metaSave(filename);
@ -969,7 +986,6 @@ public String getLinkTarget(String path) throws IOException {
@Override // DatanodeProtocol @Override // DatanodeProtocol
public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg) public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
throws IOException { throws IOException {
verifyLayoutVersion(nodeReg.getVersion());
verifySoftwareVersion(nodeReg); verifySoftwareVersion(nodeReg);
namesystem.registerDatanode(nodeReg); namesystem.registerDatanode(nodeReg);
return nodeReg; return nodeReg;
@ -1071,13 +1087,29 @@ public NamespaceInfo versionRequest() throws IOException {
* @param nodeReg node registration * @param nodeReg node registration
* @throws UnregisteredNodeException if the registration is invalid * @throws UnregisteredNodeException if the registration is invalid
*/ */
void verifyRequest(NodeRegistration nodeReg) throws IOException { private void verifyRequest(NodeRegistration nodeReg) throws IOException {
verifyLayoutVersion(nodeReg.getVersion()); // verify registration ID
if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) { final String id = nodeReg.getRegistrationID();
LOG.warn("Invalid registrationID - expected: " final String expectedID = namesystem.getRegistrationID();
+ namesystem.getRegistrationID() + " received: " if (!expectedID.equals(id)) {
+ nodeReg.getRegistrationID()); LOG.warn("Registration IDs mismatched: the "
throw new UnregisteredNodeException(nodeReg); + nodeReg.getClass().getSimpleName() + " ID is " + id
+ " but the expected ID is " + expectedID);
throw new UnregisteredNodeException(nodeReg);
}
// verify layout version if there is no rolling upgrade.
if (!namesystem.isRollingUpgrade()) {
final int lv = nodeReg.getVersion();
final int expectedLV = nodeReg instanceof NamenodeRegistration?
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION
: DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
if (expectedLV != nodeReg.getVersion()) {
LOG.warn("Layout versions mismatched: the "
+ nodeReg.getClass().getSimpleName() + " LV is " + lv
+ " but the expected LV is " + expectedLV);
throw new UnregisteredNodeException(nodeReg);
}
} }
} }
@ -1160,8 +1192,9 @@ public synchronized HAServiceStatus getServiceStatus()
* @throws IOException * @throws IOException
*/ */
void verifyLayoutVersion(int version) throws IOException { void verifyLayoutVersion(int version) throws IOException {
if (version != HdfsConstants.LAYOUT_VERSION) if (version != HdfsConstants.NAMENODE_LAYOUT_VERSION)
throw new IncorrectVersionException(version, "data node"); throw new IncorrectVersionException(
HdfsConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
} }
private void verifySoftwareVersion(DatanodeRegistration dnReg) private void verifySoftwareVersion(DatanodeRegistration dnReg)

View File

@ -445,8 +445,9 @@ public Boolean run() throws Exception {
} else { } else {
LOG.info("Image has changed. Downloading updated image from NN."); LOG.info("Image has changed. Downloading updated image from NN.");
MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage( MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
nnHostPort, sig.mostRecentCheckpointTxId, dstImage.getStorage(), true); nnHostPort, sig.mostRecentCheckpointTxId,
dstImage.saveDigestAndRenameCheckpointImage( dstImage.getStorage(), true);
dstImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE,
sig.mostRecentCheckpointTxId, downloadedHash); sig.mostRecentCheckpointTxId, downloadedHash);
} }
@ -511,8 +512,10 @@ public boolean doCheckpoint() throws IOException {
boolean loadImage = false; boolean loadImage = false;
boolean isFreshCheckpointer = (checkpointImage.getNamespaceID() == 0); boolean isFreshCheckpointer = (checkpointImage.getNamespaceID() == 0);
boolean isSameCluster = boolean isSameCluster =
(dstStorage.versionSupportsFederation() && sig.isSameCluster(checkpointImage)) || (dstStorage.versionSupportsFederation(NameNodeLayoutVersion.FEATURES)
(!dstStorage.versionSupportsFederation() && sig.namespaceIdMatches(checkpointImage)); && sig.isSameCluster(checkpointImage)) ||
(!dstStorage.versionSupportsFederation(NameNodeLayoutVersion.FEATURES)
&& sig.namespaceIdMatches(checkpointImage));
if (isFreshCheckpointer || if (isFreshCheckpointer ||
(isSameCluster && (isSameCluster &&
!sig.storageVersionMatches(checkpointImage.getStorage()))) { !sig.storageVersionMatches(checkpointImage.getStorage()))) {
@ -553,7 +556,7 @@ public boolean doCheckpoint() throws IOException {
// //
long txid = checkpointImage.getLastAppliedTxId(); long txid = checkpointImage.getLastAppliedTxId();
TransferFsImage.uploadImageFromStorage(fsName, getImageListenAddress(), TransferFsImage.uploadImageFromStorage(fsName, getImageListenAddress(),
dstStorage, txid); dstStorage, NameNodeFile.IMAGE, txid);
// error simulation code for junit test // error simulation code for junit test
CheckpointFaultInjector.getInstance().afterSecondaryUploadsNewImage(); CheckpointFaultInjector.getInstance().afterSecondaryUploadsNewImage();
@ -995,7 +998,8 @@ static void doMerge(
dstStorage.setStorageInfo(sig); dstStorage.setStorageInfo(sig);
if (loadImage) { if (loadImage) {
File file = dstStorage.findImageFile(sig.mostRecentCheckpointTxId); File file = dstStorage.findImageFile(NameNodeFile.IMAGE,
sig.mostRecentCheckpointTxId);
if (file == null) { if (file == null) {
throw new IOException("Couldn't find image file at txid " + throw new IOException("Couldn't find image file at txid " +
sig.mostRecentCheckpointTxId + " even though it should have " + sig.mostRecentCheckpointTxId + " even though it should have " +

View File

@ -23,7 +23,6 @@
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.net.HttpURLConnection; import java.net.HttpURLConnection;
import java.net.InetSocketAddress;
import java.net.URL; import java.net.URL;
import java.security.DigestInputStream; import java.security.DigestInputStream;
import java.security.MessageDigest; import java.security.MessageDigest;
@ -39,10 +38,6 @@
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.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@ -50,11 +45,13 @@
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.StorageErrorReporter; import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.hdfs.web.URLConnectionFactory; import org.apache.hadoop.hdfs.web.URLConnectionFactory;
import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -90,10 +87,9 @@ public static void downloadMostRecentImageToDirectory(URL infoServer,
null, false); null, false);
} }
public static MD5Hash downloadImageToStorage( public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId,
URL fsName, long imageTxId, Storage dstStorage, boolean needDigest) Storage dstStorage, boolean needDigest) throws IOException {
throws IOException { String fileid = GetImageServlet.getParamStringForImage(null,
String fileid = GetImageServlet.getParamStringForImage(
imageTxId, dstStorage); imageTxId, dstStorage);
String fileName = NNStorage.getCheckpointImageFileName(imageTxId); String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
@ -166,14 +162,14 @@ static void downloadEditsToStorage(URL fsName, RemoteEditLog log,
* @param myNNAddress the host/port where the local node is running an * @param myNNAddress the host/port where the local node is running an
* HTTPServer hosting GetImageServlet * HTTPServer hosting GetImageServlet
* @param storage the storage directory to transfer the image from * @param storage the storage directory to transfer the image from
* @param nnf the NameNodeFile type of the image
* @param txid the transaction ID of the image to be uploaded * @param txid the transaction ID of the image to be uploaded
*/ */
public static void uploadImageFromStorage(URL fsName, public static void uploadImageFromStorage(URL fsName, URL myNNAddress,
URL myNNAddress, Storage storage, NameNodeFile nnf, long txid) throws IOException {
Storage storage, long txid) throws IOException {
String fileid = GetImageServlet.getParamStringToPutImage( String fileid = GetImageServlet.getParamStringToPutImage(nnf, txid,
txid, myNNAddress, storage); myNNAddress, storage);
// this doesn't directly upload an image, but rather asks the NN // this doesn't directly upload an image, but rather asks the NN
// to connect back to the 2NN to download the specified image. // to connect back to the 2NN to download the specified image.
try { try {

View File

@ -38,6 +38,7 @@
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.NameNodeProxies; import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream; import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
import org.apache.hadoop.hdfs.server.namenode.FSImage; import org.apache.hadoop.hdfs.server.namenode.FSImage;
@ -45,6 +46,7 @@
import org.apache.hadoop.hdfs.server.namenode.NNStorage; import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage; import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.tools.DFSHAAdmin; import org.apache.hadoop.hdfs.tools.DFSHAAdmin;
@ -152,9 +154,9 @@ private int doRun() throws IOException {
} }
if (!checkLayoutVersion(nsInfo)) { if (!checkLayoutVersion(nsInfo)) {
LOG.fatal("Layout version on remote node (" + LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
nsInfo.getLayoutVersion() + ") does not match " + + ") does not match " + "this node's layout version ("
"this node's layout version (" + HdfsConstants.LAYOUT_VERSION + ")"); + HdfsConstants.NAMENODE_LAYOUT_VERSION + ")");
return ERR_CODE_INVALID_VERSION; return ERR_CODE_INVALID_VERSION;
} }
@ -192,7 +194,7 @@ private int doRun() throws IOException {
FSImage image = new FSImage(conf); FSImage image = new FSImage(conf);
try { try {
image.getStorage().setStorageInfo(storage); image.getStorage().setStorageInfo(storage);
image.initEditLog(); image.initEditLog(StartupOption.REGULAR);
assert image.getEditLog().isOpenForRead() : assert image.getEditLog().isOpenForRead() :
"Expected edit log to be open for read"; "Expected edit log to be open for read";
@ -206,9 +208,9 @@ private int doRun() throws IOException {
// Download that checkpoint into our storage directories. // Download that checkpoint into our storage directories.
MD5Hash hash = TransferFsImage.downloadImageToStorage( MD5Hash hash = TransferFsImage.downloadImageToStorage(
otherHttpAddr, imageTxId, otherHttpAddr, imageTxId, storage, true);
storage, true); image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId,
image.saveDigestAndRenameCheckpointImage(imageTxId, hash); hash);
} catch (IOException ioe) { } catch (IOException ioe) {
image.close(); image.close();
throw ioe; throw ioe;
@ -256,7 +258,7 @@ private boolean checkLogsAvailableForRead(FSImage image, long imageTxId,
} }
private boolean checkLayoutVersion(NamespaceInfo nsInfo) throws IOException { private boolean checkLayoutVersion(NamespaceInfo nsInfo) throws IOException {
return (nsInfo.getLayoutVersion() == HdfsConstants.LAYOUT_VERSION); return (nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION);
} }
private void parseConfAndFindOtherNN() throws IOException { private void parseConfAndFindOtherNN() throws IOException {

View File

@ -224,7 +224,7 @@ void doTailEdits() throws IOException, InterruptedException {
// disk are ignored. // disk are ignored.
long editsLoaded = 0; long editsLoaded = 0;
try { try {
editsLoaded = image.loadEdits(streams, namesystem, null); editsLoaded = image.loadEdits(streams, namesystem);
} catch (EditLogInputException elie) { } catch (EditLogInputException elie) {
editsLoaded = elie.getNumEditsLoaded(); editsLoaded = elie.getNumEditsLoaded();
throw elie; throw elie;

View File

@ -40,6 +40,7 @@
import org.apache.hadoop.hdfs.server.namenode.CheckpointConf; import org.apache.hadoop.hdfs.server.namenode.CheckpointConf;
import org.apache.hadoop.hdfs.server.namenode.FSImage; import org.apache.hadoop.hdfs.server.namenode.FSImage;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceCancelledException; import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceCancelledException;
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage; import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
@ -141,9 +142,14 @@ public void stop() throws IOException {
} }
} }
public void triggerRollbackCheckpoint() {
thread.interrupt();
}
private void doCheckpoint() throws InterruptedException, IOException { private void doCheckpoint() throws InterruptedException, IOException {
assert canceler != null; assert canceler != null;
final long txid; final long txid;
final NameNodeFile imageType;
namesystem.longReadLockInterruptibly(); namesystem.longReadLockInterruptibly();
try { try {
@ -163,7 +169,15 @@ private void doCheckpoint() throws InterruptedException, IOException {
return; return;
} }
img.saveNamespace(namesystem, canceler); if (namesystem.isRollingUpgrade()
&& !namesystem.getFSImage().hasRollbackFSImage()) {
// if we will do rolling upgrade but have not created the rollback image
// yet, name this checkpoint as fsimage_rollback
imageType = NameNodeFile.IMAGE_ROLLBACK;
} else {
imageType = NameNodeFile.IMAGE;
}
img.saveNamespace(namesystem, imageType, canceler);
txid = img.getStorage().getMostRecentCheckpointTxId(); txid = img.getStorage().getMostRecentCheckpointTxId();
assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" + assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +
thisCheckpointTxId + " but instead saved at txid=" + txid; thisCheckpointTxId + " but instead saved at txid=" + txid;
@ -179,9 +193,8 @@ private void doCheckpoint() throws InterruptedException, IOException {
Future<Void> upload = executor.submit(new Callable<Void>() { Future<Void> upload = executor.submit(new Callable<Void>() {
@Override @Override
public Void call() throws IOException { public Void call() throws IOException {
TransferFsImage.uploadImageFromStorage( TransferFsImage.uploadImageFromStorage(activeNNAddress, myNNAddress,
activeNNAddress, myNNAddress, namesystem.getFSImage().getStorage(), imageType, txid);
namesystem.getFSImage().getStorage(), txid);
return null; return null;
} }
}); });
@ -266,16 +279,20 @@ private void preventCheckpointsFor(long delayMs) {
} }
private void doWork() { private void doWork() {
final long checkPeriod = 1000 * checkpointConf.getCheckPeriod();
// Reset checkpoint time so that we don't always checkpoint // Reset checkpoint time so that we don't always checkpoint
// on startup. // on startup.
lastCheckpointTime = now(); lastCheckpointTime = now();
while (shouldRun) { while (shouldRun) {
try { boolean needRollbackCheckpoint = namesystem.isNeedRollbackFsImage();
Thread.sleep(1000 * checkpointConf.getCheckPeriod()); if (!needRollbackCheckpoint) {
} catch (InterruptedException ie) { try {
} Thread.sleep(checkPeriod);
if (!shouldRun) { } catch (InterruptedException ie) {
break; }
if (!shouldRun) {
break;
}
} }
try { try {
// We may have lost our ticket since last checkpoint, log in again, just in case // We may have lost our ticket since last checkpoint, log in again, just in case
@ -287,8 +304,10 @@ private void doWork() {
long uncheckpointed = countUncheckpointedTxns(); long uncheckpointed = countUncheckpointedTxns();
long secsSinceLast = (now - lastCheckpointTime)/1000; long secsSinceLast = (now - lastCheckpointTime)/1000;
boolean needCheckpoint = false; boolean needCheckpoint = needRollbackCheckpoint;
if (uncheckpointed >= checkpointConf.getTxnCount()) { if (needCheckpoint) {
LOG.info("Triggering a rollback fsimage for rolling upgrade.");
} else if (uncheckpointed >= checkpointConf.getTxnCount()) {
LOG.info("Triggering checkpoint because there have been " + LOG.info("Triggering checkpoint because there have been " +
uncheckpointed + " txns since the last checkpoint, which " + uncheckpointed + " txns since the last checkpoint, which " +
"exceeds the configured threshold " + "exceeds the configured threshold " +
@ -313,6 +332,13 @@ private void doWork() {
if (needCheckpoint) { if (needCheckpoint) {
doCheckpoint(); doCheckpoint();
// reset needRollbackCheckpoint to false only when we finish a ckpt
// for rollback image
if (needRollbackCheckpoint
&& namesystem.getFSImage().hasRollbackFSImage()) {
namesystem.setCreatedRollbackImages(true);
namesystem.setNeedRollbackFsImage(false);
}
lastCheckpointTime = now; lastCheckpointTime = now;
} }
} catch (SaveNamespaceCancelledException ce) { } catch (SaveNamespaceCancelledException ce) {

View File

@ -19,6 +19,7 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
@ -31,11 +32,14 @@ public class HeartbeatResponse {
/** Information about the current HA-related state of the NN */ /** Information about the current HA-related state of the NN */
private NNHAStatusHeartbeat haStatus; private NNHAStatusHeartbeat haStatus;
private RollingUpgradeStatus rollingUpdateStatus;
public HeartbeatResponse(DatanodeCommand[] cmds, public HeartbeatResponse(DatanodeCommand[] cmds,
NNHAStatusHeartbeat haStatus) { NNHAStatusHeartbeat haStatus, RollingUpgradeStatus rollingUpdateStatus) {
commands = cmds; commands = cmds;
this.haStatus = haStatus; this.haStatus = haStatus;
this.rollingUpdateStatus = rollingUpdateStatus;
} }
public DatanodeCommand[] getCommands() { public DatanodeCommand[] getCommands() {
@ -45,4 +49,8 @@ public DatanodeCommand[] getCommands() {
public NNHAStatusHeartbeat getNameNodeHaState() { public NNHAStatusHeartbeat getNameNodeHaState() {
return haStatus; return haStatus;
} }
public RollingUpgradeStatus getRollingUpdateStatus() {
return rollingUpdateStatus;
}
} }

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage; import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
@ -41,13 +42,14 @@ public class NamespaceInfo extends StorageInfo {
String softwareVersion; String softwareVersion;
public NamespaceInfo() { public NamespaceInfo() {
super(); super(NodeType.NAME_NODE);
buildVersion = null; buildVersion = null;
} }
public NamespaceInfo(int nsID, String clusterID, String bpID, public NamespaceInfo(int nsID, String clusterID, String bpID,
long cT, String buildVersion, String softwareVersion) { long cT, String buildVersion, String softwareVersion) {
super(HdfsConstants.LAYOUT_VERSION, nsID, clusterID, cT); super(HdfsConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
NodeType.NAME_NODE);
blockPoolID = bpID; blockPoolID = bpID;
this.buildVersion = buildVersion; this.buildVersion = buildVersion;
this.softwareVersion = softwareVersion; this.softwareVersion = softwareVersion;

View File

@ -19,6 +19,7 @@
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.URL; import java.net.URL;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
@ -47,9 +48,12 @@
import org.apache.hadoop.hdfs.NameNodeProxies; import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol; import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotException; import org.apache.hadoop.hdfs.protocol.SnapshotException;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage; import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
@ -64,6 +68,8 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import com.google.common.base.Preconditions;
/** /**
* This class provides some DFS administrative access shell commands. * This class provides some DFS administrative access shell commands.
*/ */
@ -271,7 +277,71 @@ public void run(Path path) throws IOException {
dfs.setQuota(path, HdfsConstants.QUOTA_DONT_SET, quota); dfs.setQuota(path, HdfsConstants.QUOTA_DONT_SET, quota);
} }
} }
private static class RollingUpgradeCommand {
static final String NAME = "rollingUpgrade";
static final String USAGE = "-"+NAME+" [<query|prepare|finalize>]";
static final String DESCRIPTION = USAGE + ":\n"
+ " query: query the current rolling upgrade status.\n"
+ " prepare: prepare a new rolling upgrade."
+ " finalize: finalize the current rolling upgrade.";
/** Check if a command is the rollingUpgrade command
*
* @param cmd A string representation of a command starting with "-"
* @return true if this is a clrQuota command; false otherwise
*/
static boolean matches(String cmd) {
return ("-"+NAME).equals(cmd);
}
private static void printMessage(RollingUpgradeInfo info,
PrintStream out) {
if (info != null && info.isStarted()) {
if (!info.createdRollbackImages()) {
out.println(
"Preparing for upgrade. Data is being saved for rollback."
+ "\nRun \"dfsadmin -rollingUpgrade query\" to check the status"
+ "\nfor proceeding with rolling upgrade");
out.println(info);
} else if (!info.isFinalized()) {
out.println("Proceed with rolling upgrade:");
out.println(info);
} else {
out.println("Rolling upgrade is finalized.");
out.println(info);
}
} else {
out.println("There is no rolling upgrade in progress.");
}
}
static int run(DistributedFileSystem dfs, String[] argv, int idx) throws IOException {
final RollingUpgradeAction action = RollingUpgradeAction.fromString(
argv.length >= 2? argv[1]: "");
if (action == null) {
throw new IllegalArgumentException("Failed to covert \"" + argv[1]
+"\" to " + RollingUpgradeAction.class.getSimpleName());
}
System.out.println(action + " rolling upgrade ...");
final RollingUpgradeInfo info = dfs.rollingUpgrade(action);
switch(action){
case QUERY:
break;
case PREPARE:
Preconditions.checkState(info.isStarted());
break;
case FINALIZE:
Preconditions.checkState(info.isFinalized());
break;
}
printMessage(info, System.out);
return 0;
}
}
/** /**
* Construct a DFSAdmin object. * Construct a DFSAdmin object.
*/ */
@ -576,9 +646,11 @@ private void printHelp(String cmd) {
"\t[" + ClearQuotaCommand.USAGE +"]\n" + "\t[" + ClearQuotaCommand.USAGE +"]\n" +
"\t[" + SetSpaceQuotaCommand.USAGE + "]\n" + "\t[" + SetSpaceQuotaCommand.USAGE + "]\n" +
"\t[" + ClearSpaceQuotaCommand.USAGE +"]\n" + "\t[" + ClearSpaceQuotaCommand.USAGE +"]\n" +
"\t[-finalizeUpgrade]\n" +
"\t[" + RollingUpgradeCommand.USAGE +"]\n" +
"\t[-refreshServiceAcl]\n" + "\t[-refreshServiceAcl]\n" +
"\t[-refreshUserToGroupsMappings]\n" + "\t[-refreshUserToGroupsMappings]\n" +
"\t[refreshSuperUserGroupsConfiguration]\n" + "\t[-refreshSuperUserGroupsConfiguration]\n" +
"\t[-refreshCallQueue]\n" + "\t[-refreshCallQueue]\n" +
"\t[-printTopology]\n" + "\t[-printTopology]\n" +
"\t[-refreshNamenodes datanodehost:port]\n"+ "\t[-refreshNamenodes datanodehost:port]\n"+
@ -587,6 +659,8 @@ private void printHelp(String cmd) {
"\t[-fetchImage <local directory>]\n" + "\t[-fetchImage <local directory>]\n" +
"\t[-allowSnapshot <snapshotDir>]\n" + "\t[-allowSnapshot <snapshotDir>]\n" +
"\t[-disallowSnapshot <snapshotDir>]\n" + "\t[-disallowSnapshot <snapshotDir>]\n" +
"\t[-shutdownDatanode <datanode_host:ipc_port> [upgrade]]\n" +
"\t[-getDatanodeInfo <datanode_host:ipc_port>\n" +
"\t[-help [cmd]]\n"; "\t[-help [cmd]]\n";
String report ="-report: \tReports basic filesystem information and statistics.\n"; String report ="-report: \tReports basic filesystem information and statistics.\n";
@ -685,6 +759,18 @@ private void printHelp(String cmd) {
String disallowSnapshot = "-disallowSnapshot <snapshotDir>:\n" + String disallowSnapshot = "-disallowSnapshot <snapshotDir>:\n" +
"\tDo not allow snapshots to be taken on a directory any more.\n"; "\tDo not allow snapshots to be taken on a directory any more.\n";
String shutdownDatanode = "-shutdownDatanode <datanode_host:ipc_port> [upgrade]\n"
+ "\tSubmit a shutdown request for the given datanode. If an optional\n"
+ "\t\"upgrade\" argument is specified, clients accessing the datanode\n"
+ "\twill be advised to wait for it to restart and the fast start-up\n"
+ "\tmode will be enabled. When the restart does not happen in time,\n"
+ "\tclients will timeout and ignore the datanode. In such case, the\n"
+ "\tfast start-up mode will also be disabled.\n";
String getDatanodeInfo = "-getDatanodeInfo <datanode_host:ipc_port>\n"
+ "\tGet the information about the given datanode. This command can\n"
+ "\tbe used for checking if a datanode is alive.\n";
String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" + String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
"\t\tis specified.\n"; "\t\tis specified.\n";
@ -703,6 +789,8 @@ private void printHelp(String cmd) {
System.out.println(refreshNodes); System.out.println(refreshNodes);
} else if ("finalizeUpgrade".equals(cmd)) { } else if ("finalizeUpgrade".equals(cmd)) {
System.out.println(finalizeUpgrade); System.out.println(finalizeUpgrade);
} else if (RollingUpgradeCommand.matches("-"+cmd)) {
System.out.println(RollingUpgradeCommand.DESCRIPTION);
} else if ("metasave".equals(cmd)) { } else if ("metasave".equals(cmd)) {
System.out.println(metaSave); System.out.println(metaSave);
} else if (SetQuotaCommand.matches("-"+cmd)) { } else if (SetQuotaCommand.matches("-"+cmd)) {
@ -735,6 +823,10 @@ private void printHelp(String cmd) {
System.out.println(allowSnapshot); System.out.println(allowSnapshot);
} else if ("disallowSnapshot".equalsIgnoreCase(cmd)) { } else if ("disallowSnapshot".equalsIgnoreCase(cmd)) {
System.out.println(disallowSnapshot); System.out.println(disallowSnapshot);
} else if ("shutdownDatanode".equalsIgnoreCase(cmd)) {
System.out.println(shutdownDatanode);
} else if ("getDatanodeInfo".equalsIgnoreCase(cmd)) {
System.out.println(getDatanodeInfo);
} else if ("help".equals(cmd)) { } else if ("help".equals(cmd)) {
System.out.println(help); System.out.println(help);
} else { } else {
@ -746,6 +838,7 @@ private void printHelp(String cmd) {
System.out.println(restoreFailedStorage); System.out.println(restoreFailedStorage);
System.out.println(refreshNodes); System.out.println(refreshNodes);
System.out.println(finalizeUpgrade); System.out.println(finalizeUpgrade);
System.out.println(RollingUpgradeCommand.DESCRIPTION);
System.out.println(metaSave); System.out.println(metaSave);
System.out.println(SetQuotaCommand.DESCRIPTION); System.out.println(SetQuotaCommand.DESCRIPTION);
System.out.println(ClearQuotaCommand.DESCRIPTION); System.out.println(ClearQuotaCommand.DESCRIPTION);
@ -762,6 +855,8 @@ private void printHelp(String cmd) {
System.out.println(fetchImage); System.out.println(fetchImage);
System.out.println(allowSnapshot); System.out.println(allowSnapshot);
System.out.println(disallowSnapshot); System.out.println(disallowSnapshot);
System.out.println(shutdownDatanode);
System.out.println(getDatanodeInfo);
System.out.println(help); System.out.println(help);
System.out.println(); System.out.println();
ToolRunner.printGenericCommandUsage(System.out); ToolRunner.printGenericCommandUsage(System.out);
@ -980,6 +1075,9 @@ private static void printUsage(String cmd) {
} else if ("-finalizeUpgrade".equals(cmd)) { } else if ("-finalizeUpgrade".equals(cmd)) {
System.err.println("Usage: java DFSAdmin" System.err.println("Usage: java DFSAdmin"
+ " [-finalizeUpgrade]"); + " [-finalizeUpgrade]");
} else if (RollingUpgradeCommand.matches(cmd)) {
System.err.println("Usage: java DFSAdmin"
+ " [" + RollingUpgradeCommand.USAGE+"]");
} else if ("-metasave".equals(cmd)) { } else if ("-metasave".equals(cmd)) {
System.err.println("Usage: java DFSAdmin" System.err.println("Usage: java DFSAdmin"
+ " [-metasave filename]"); + " [-metasave filename]");
@ -1034,6 +1132,7 @@ private static void printUsage(String cmd) {
System.err.println(" [-restoreFailedStorage true|false|check]"); System.err.println(" [-restoreFailedStorage true|false|check]");
System.err.println(" [-refreshNodes]"); System.err.println(" [-refreshNodes]");
System.err.println(" [-finalizeUpgrade]"); System.err.println(" [-finalizeUpgrade]");
System.err.println(" ["+RollingUpgradeCommand.USAGE+"]");
System.err.println(" [-metasave filename]"); System.err.println(" [-metasave filename]");
System.err.println(" [-refreshServiceAcl]"); System.err.println(" [-refreshServiceAcl]");
System.err.println(" [-refreshUserToGroupsMappings]"); System.err.println(" [-refreshUserToGroupsMappings]");
@ -1048,6 +1147,8 @@ private static void printUsage(String cmd) {
System.err.println(" ["+ClearSpaceQuotaCommand.USAGE+"]"); System.err.println(" ["+ClearSpaceQuotaCommand.USAGE+"]");
System.err.println(" [-setBalancerBandwidth <bandwidth in bytes per second>]"); System.err.println(" [-setBalancerBandwidth <bandwidth in bytes per second>]");
System.err.println(" [-fetchImage <local directory>]"); System.err.println(" [-fetchImage <local directory>]");
System.err.println(" [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]");
System.err.println(" [-getDatanodeInfo <datanode_host:ipc_port>]");
System.err.println(" [-help [cmd]]"); System.err.println(" [-help [cmd]]");
System.err.println(); System.err.println();
ToolRunner.printGenericCommandUsage(System.err); ToolRunner.printGenericCommandUsage(System.err);
@ -1119,6 +1220,11 @@ public int run(String[] argv) throws Exception {
printUsage(cmd); printUsage(cmd);
return exitCode; return exitCode;
} }
} else if (RollingUpgradeCommand.matches(cmd)) {
if (argv.length < 1 || argv.length > 2) {
printUsage(cmd);
return exitCode;
}
} else if ("-metasave".equals(cmd)) { } else if ("-metasave".equals(cmd)) {
if (argv.length != 2) { if (argv.length != 2) {
printUsage(cmd); printUsage(cmd);
@ -1159,6 +1265,16 @@ public int run(String[] argv) throws Exception {
printUsage(cmd); printUsage(cmd);
return exitCode; return exitCode;
} }
} else if ("-shutdownDatanode".equals(cmd)) {
if ((argv.length != 2) && (argv.length != 3)) {
printUsage(cmd);
return exitCode;
}
} else if ("-getDatanodeInfo".equals(cmd)) {
if (argv.length != 2) {
printUsage(cmd);
return exitCode;
}
} }
// initialize DFSAdmin // initialize DFSAdmin
@ -1194,6 +1310,8 @@ public int run(String[] argv) throws Exception {
exitCode = refreshNodes(); exitCode = refreshNodes();
} else if ("-finalizeUpgrade".equals(cmd)) { } else if ("-finalizeUpgrade".equals(cmd)) {
exitCode = finalizeUpgrade(); exitCode = finalizeUpgrade();
} else if (RollingUpgradeCommand.matches(cmd)) {
exitCode = RollingUpgradeCommand.run(getDFS(), argv, i);
} else if ("-metasave".equals(cmd)) { } else if ("-metasave".equals(cmd)) {
exitCode = metaSave(argv, i); exitCode = metaSave(argv, i);
} else if (ClearQuotaCommand.matches(cmd)) { } else if (ClearQuotaCommand.matches(cmd)) {
@ -1222,6 +1340,10 @@ public int run(String[] argv) throws Exception {
exitCode = setBalancerBandwidth(argv, i); exitCode = setBalancerBandwidth(argv, i);
} else if ("-fetchImage".equals(cmd)) { } else if ("-fetchImage".equals(cmd)) {
exitCode = fetchImage(argv, i); exitCode = fetchImage(argv, i);
} else if ("-shutdownDatanode".equals(cmd)) {
exitCode = shutdownDatanode(argv, i);
} else if ("-getDatanodeInfo".equals(cmd)) {
exitCode = getDatanodeInfo(argv, i);
} else if ("-help".equals(cmd)) { } else if ("-help".equals(cmd)) {
if (i < argv.length) { if (i < argv.length) {
printHelp(argv[i]); printHelp(argv[i]);
@ -1306,6 +1428,35 @@ private int refreshNamenodes(String[] argv, int i) throws IOException {
return 0; return 0;
} }
private int shutdownDatanode(String[] argv, int i) throws IOException {
final String dn = argv[i];
ClientDatanodeProtocol dnProxy = getDataNodeProxy(dn);
boolean upgrade = false;
if (argv.length-1 == i+1) {
if ("upgrade".equalsIgnoreCase(argv[i+1])) {
upgrade = true;
} else {
printUsage("-shutdownDatanode");
return -1;
}
}
dnProxy.shutdownDatanode(upgrade);
System.out.println("Submitted a shutdown request to datanode " + dn);
return 0;
}
private int getDatanodeInfo(String[] argv, int i) throws IOException {
ClientDatanodeProtocol dnProxy = getDataNodeProxy(argv[i]);
try {
DatanodeLocalInfo dnInfo = dnProxy.getDatanodeInfo();
System.out.println(dnInfo.getDatanodeLocalReport());
} catch (IOException ioe) {
System.err.println("Datanode unreachable.");
return -1;
}
return 0;
}
/** /**
* main() has some simple utility methods. * main() has some simple utility methods.
* @param argv Command line parameters. * @param argv Command line parameters.

View File

@ -30,10 +30,10 @@
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
import org.apache.hadoop.hdfs.protocol.LayoutFlags; import org.apache.hadoop.hdfs.protocol.LayoutFlags;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization; import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
import org.apache.hadoop.hdfs.server.namenode.INodeId; import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement; import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.WritableUtils;
@ -158,7 +158,8 @@ public void loadImage(DataInputStream in, ImageVisitor v,
imageVersion = in.readInt(); imageVersion = in.readInt();
if( !canLoadVersion(imageVersion)) if( !canLoadVersion(imageVersion))
throw new IOException("Cannot process fslayout version " + imageVersion); throw new IOException("Cannot process fslayout version " + imageVersion);
if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
LayoutFlags.read(in); LayoutFlags.read(in);
} }
@ -169,22 +170,25 @@ public void loadImage(DataInputStream in, ImageVisitor v,
v.visit(ImageElement.GENERATION_STAMP, in.readLong()); v.visit(ImageElement.GENERATION_STAMP, in.readLong());
if (LayoutVersion.supports(Feature.SEQUENTIAL_BLOCK_ID, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imageVersion)) {
v.visit(ImageElement.GENERATION_STAMP_V2, in.readLong()); v.visit(ImageElement.GENERATION_STAMP_V2, in.readLong());
v.visit(ImageElement.GENERATION_STAMP_V1_LIMIT, in.readLong()); v.visit(ImageElement.GENERATION_STAMP_V1_LIMIT, in.readLong());
v.visit(ImageElement.LAST_ALLOCATED_BLOCK_ID, in.readLong()); v.visit(ImageElement.LAST_ALLOCATED_BLOCK_ID, in.readLong());
} }
if (LayoutVersion.supports(Feature.STORED_TXIDS, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.STORED_TXIDS, imageVersion)) {
v.visit(ImageElement.TRANSACTION_ID, in.readLong()); v.visit(ImageElement.TRANSACTION_ID, in.readLong());
} }
if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, imageVersion)) {
v.visit(ImageElement.LAST_INODE_ID, in.readLong()); v.visit(ImageElement.LAST_INODE_ID, in.readLong());
} }
boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT, boolean supportSnapshot = NameNodeLayoutVersion.supports(
imageVersion); LayoutVersion.Feature.SNAPSHOT, imageVersion);
if (supportSnapshot) { if (supportSnapshot) {
v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt()); v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt());
int numSnapshots = in.readInt(); int numSnapshots = in.readInt();
@ -194,7 +198,8 @@ public void loadImage(DataInputStream in, ImageVisitor v,
} }
} }
if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FSIMAGE_COMPRESSION, imageVersion)) {
boolean isCompressed = in.readBoolean(); boolean isCompressed = in.readBoolean();
v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed)); v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
if (isCompressed) { if (isCompressed) {
@ -216,11 +221,13 @@ public void loadImage(DataInputStream in, ImageVisitor v,
processINodesUC(in, v, skipBlocks); processINodesUC(in, v, skipBlocks);
if (LayoutVersion.supports(Feature.DELEGATION_TOKEN, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.DELEGATION_TOKEN, imageVersion)) {
processDelegationTokens(in, v); processDelegationTokens(in, v);
} }
if (LayoutVersion.supports(Feature.CACHING, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.CACHING, imageVersion)) {
processCacheManagerState(in, v); processCacheManagerState(in, v);
} }
v.leaveEnclosingElement(); // FSImage v.leaveEnclosingElement(); // FSImage
@ -323,7 +330,8 @@ private void processINodesUC(DataInputStream in, ImageVisitor v,
String n = new String(name, "UTF8"); String n = new String(name, "UTF8");
v.visit(ImageElement.INODE_PATH, n); v.visit(ImageElement.INODE_PATH, n);
if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, imageVersion)) {
long inodeId = in.readLong(); long inodeId = in.readLong();
v.visit(ImageElement.INODE_ID, inodeId); v.visit(ImageElement.INODE_ID, inodeId);
} }
@ -443,7 +451,8 @@ private void processINodes(DataInputStream in, ImageVisitor v,
v.visitEnclosingElement(ImageElement.INODES, v.visitEnclosingElement(ImageElement.INODES,
ImageElement.NUM_INODES, numInodes); ImageElement.NUM_INODES, numInodes);
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
if (!supportSnapshot) { if (!supportSnapshot) {
processLocalNameINodes(in, v, numInodes, skipBlocks); processLocalNameINodes(in, v, numInodes, skipBlocks);
} else { } else {
@ -584,7 +593,8 @@ private void processDirectoryDiff(DataInputStream in, ImageVisitor v,
if (!useRoot) { if (!useRoot) {
if (in.readBoolean()) { if (in.readBoolean()) {
v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_DIRECTORY_ATTRIBUTES); v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_DIRECTORY_ATTRIBUTES);
if (LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
processINodeDirectoryAttributes(in, v, currentINodeName); processINodeDirectoryAttributes(in, v, currentINodeName);
} else { } else {
processINode(in, v, true, currentINodeName, true); processINode(in, v, true, currentINodeName, true);
@ -678,10 +688,10 @@ private String readINodePath(DataInputStream in, String parentName)
private void processINode(DataInputStream in, ImageVisitor v, private void processINode(DataInputStream in, ImageVisitor v,
boolean skipBlocks, String parentName, boolean isSnapshotCopy) boolean skipBlocks, String parentName, boolean isSnapshotCopy)
throws IOException { throws IOException {
boolean supportSnapshot = boolean supportSnapshot = NameNodeLayoutVersion.supports(
LayoutVersion.supports(Feature.SNAPSHOT, imageVersion); LayoutVersion.Feature.SNAPSHOT, imageVersion);
boolean supportInodeId = boolean supportInodeId = NameNodeLayoutVersion.supports(
LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion); LayoutVersion.Feature.ADD_INODE_ID, imageVersion);
v.visitEnclosingElement(ImageElement.INODE); v.visitEnclosingElement(ImageElement.INODE);
final String pathName = readINodePath(in, parentName); final String pathName = readINodePath(in, parentName);
@ -694,7 +704,8 @@ private void processINode(DataInputStream in, ImageVisitor v,
} }
v.visit(ImageElement.REPLICATION, in.readShort()); v.visit(ImageElement.REPLICATION, in.readShort());
v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong())); v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion)) if(NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FILE_ACCESS_TIME, imageVersion))
v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong())); v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
v.visit(ImageElement.BLOCK_SIZE, in.readLong()); v.visit(ImageElement.BLOCK_SIZE, in.readLong());
int numBlocks = in.readInt(); int numBlocks = in.readInt();
@ -723,7 +734,8 @@ private void processINode(DataInputStream in, ImageVisitor v,
dirNodeMap.put(inodeId, pathName); dirNodeMap.put(inodeId, pathName);
} }
v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1); v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion)) if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.DISKSPACE_QUOTA, imageVersion))
v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1); v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
if (supportSnapshot) { if (supportSnapshot) {
boolean snapshottable = in.readBoolean(); boolean snapshottable = in.readBoolean();
@ -771,7 +783,8 @@ private void processINodeFileAttributes(DataInputStream in, ImageVisitor v,
v.visit(ImageElement.INODE_PATH, pathName); v.visit(ImageElement.INODE_PATH, pathName);
processPermission(in, v); processPermission(in, v);
v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong())); v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion)) { if(NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FILE_ACCESS_TIME, imageVersion)) {
v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong())); v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
} }
@ -800,7 +813,8 @@ private void processFileDiff(DataInputStream in, ImageVisitor v,
v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong()); v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong());
if (in.readBoolean()) { if (in.readBoolean()) {
v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_FILE_ATTRIBUTES); v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_FILE_ATTRIBUTES);
if (LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) { if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
processINodeFileAttributes(in, v, currentINodeName); processINodeFileAttributes(in, v, currentINodeName);
} else { } else {
processINode(in, v, true, currentINodeName, true); processINode(in, v, true, currentINodeName, true);

View File

@ -20,6 +20,7 @@
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.File; import java.io.File;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.InputStreamReader; import java.io.InputStreamReader;
@ -65,23 +66,17 @@ public static void verifySavedMD5(File dataFile, MD5Hash expectedMD5)
} }
/** /**
* Read the md5 checksum stored alongside the given file, or null * Read the md5 file stored alongside the given data file
* if no md5 is stored. * and match the md5 file content.
* @param dataFile the file containing data * @param dataFile the file containing data
* @return the checksum stored in dataFile.md5 * @return a matcher with two matched groups
* where group(1) is the md5 string and group(2) is the data file path.
*/ */
public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException { private static Matcher readStoredMd5(File md5File) throws IOException {
File md5File = getDigestFileForFile(dataFile);
String md5Line;
if (!md5File.exists()) {
return null;
}
BufferedReader reader = BufferedReader reader =
new BufferedReader(new InputStreamReader(new FileInputStream( new BufferedReader(new InputStreamReader(new FileInputStream(
md5File), Charsets.UTF_8)); md5File), Charsets.UTF_8));
String md5Line;
try { try {
md5Line = reader.readLine(); md5Line = reader.readLine();
if (md5Line == null) { md5Line = ""; } if (md5Line == null) { md5Line = ""; }
@ -94,9 +89,24 @@ public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException {
Matcher matcher = LINE_REGEX.matcher(md5Line); Matcher matcher = LINE_REGEX.matcher(md5Line);
if (!matcher.matches()) { if (!matcher.matches()) {
throw new IOException("Invalid MD5 file at " + md5File throw new IOException("Invalid MD5 file " + md5File + ": the content \""
+ " (does not match expected pattern)"); + md5Line + "\" does not match the expected pattern.");
} }
return matcher;
}
/**
* Read the md5 checksum stored alongside the given data file.
* @param dataFile the file containing data
* @return the checksum stored in dataFile.md5
*/
public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException {
final File md5File = getDigestFileForFile(dataFile);
if (!md5File.exists()) {
return null;
}
final Matcher matcher = readStoredMd5(md5File);
String storedHash = matcher.group(1); String storedHash = matcher.group(1);
File referencedFile = new File(matcher.group(2)); File referencedFile = new File(matcher.group(2));
@ -135,15 +145,37 @@ public static MD5Hash computeMd5ForFile(File dataFile) throws IOException {
*/ */
public static void saveMD5File(File dataFile, MD5Hash digest) public static void saveMD5File(File dataFile, MD5Hash digest)
throws IOException { throws IOException {
final String digestString = StringUtils.byteToHexString(digest.getDigest());
saveMD5File(dataFile, digestString);
}
private static void saveMD5File(File dataFile, String digestString)
throws IOException {
File md5File = getDigestFileForFile(dataFile); File md5File = getDigestFileForFile(dataFile);
String digestString = StringUtils.byteToHexString(
digest.getDigest());
String md5Line = digestString + " *" + dataFile.getName() + "\n"; String md5Line = digestString + " *" + dataFile.getName() + "\n";
AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File); AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File);
afos.write(md5Line.getBytes(Charsets.UTF_8)); afos.write(md5Line.getBytes(Charsets.UTF_8));
afos.close(); afos.close();
LOG.debug("Saved MD5 " + digest + " to " + md5File);
if (LOG.isDebugEnabled()) {
LOG.debug("Saved MD5 " + digestString + " to " + md5File);
}
}
public static void renameMD5File(File oldDataFile, File newDataFile)
throws IOException {
final File fromFile = getDigestFileForFile(oldDataFile);
if (!fromFile.exists()) {
throw new FileNotFoundException(fromFile + " does not exist.");
}
final String digestString = readStoredMd5(fromFile).group(1);
saveMD5File(newDataFile, digestString);
if (!fromFile.delete()) {
LOG.warn("deleting " + fromFile.getAbsolutePath() + " FAILED");
}
} }
/** /**

View File

@ -127,6 +127,28 @@ message GetHdfsBlockLocationsResponseProto {
repeated uint32 volumeIndexes = 2 [ packed = true ]; repeated uint32 volumeIndexes = 2 [ packed = true ];
} }
/**
* forUpgrade - if true, clients are advised to wait for restart and quick
* upgrade restart is instrumented. Otherwise, datanode does
* the regular shutdown.
*/
message ShutdownDatanodeRequestProto {
required bool forUpgrade = 1;
}
message ShutdownDatanodeResponseProto {
}
/**
* Ping datanode for liveness and quick info
*/
message GetDatanodeInfoRequestProto {
}
message GetDatanodeInfoResponseProto {
required DatanodeLocalInfoProto localInfo = 1;
}
/** /**
* Protocol used from client to the Datanode. * Protocol used from client to the Datanode.
* See the request and response for details of rpc call. * See the request and response for details of rpc call.
@ -164,4 +186,10 @@ service ClientDatanodeProtocolService {
*/ */
rpc getHdfsBlockLocations(GetHdfsBlockLocationsRequestProto) rpc getHdfsBlockLocations(GetHdfsBlockLocationsRequestProto)
returns(GetHdfsBlockLocationsResponseProto); returns(GetHdfsBlockLocationsResponseProto);
rpc shutdownDatanode(ShutdownDatanodeRequestProto)
returns(ShutdownDatanodeResponseProto);
rpc getDatanodeInfo(GetDatanodeInfoRequestProto)
returns(GetDatanodeInfoResponseProto);
} }

View File

@ -333,6 +333,27 @@ message FinalizeUpgradeRequestProto { // no parameters
message FinalizeUpgradeResponseProto { // void response message FinalizeUpgradeResponseProto { // void response
} }
enum RollingUpgradeActionProto {
QUERY = 1;
START = 2;
FINALIZE = 3;
}
message RollingUpgradeRequestProto {
required RollingUpgradeActionProto action = 1;
}
message RollingUpgradeInfoProto {
required RollingUpgradeStatusProto status = 1;
required uint64 startTime = 2;
required uint64 finalizeTime = 3;
required bool createdRollbackImages = 4;
}
message RollingUpgradeResponseProto {
optional RollingUpgradeInfoProto rollingUpgradeInfo= 1;
}
message ListCorruptFileBlocksRequestProto { message ListCorruptFileBlocksRequestProto {
required string path = 1; required string path = 1;
optional string cookie = 2; optional string cookie = 2;
@ -659,6 +680,8 @@ service ClientNamenodeProtocol {
rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto); rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
rpc finalizeUpgrade(FinalizeUpgradeRequestProto) rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
returns(FinalizeUpgradeResponseProto); returns(FinalizeUpgradeResponseProto);
rpc rollingUpgrade(RollingUpgradeRequestProto)
returns(RollingUpgradeResponseProto);
rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto) rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
returns(ListCorruptFileBlocksResponseProto); returns(ListCorruptFileBlocksResponseProto);
rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto); rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);

View File

@ -224,6 +224,7 @@ message NNHAStatusHeartbeatProto {
message HeartbeatResponseProto { message HeartbeatResponseProto {
repeated DatanodeCommandProto cmds = 1; // Returned commands can be null repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
required NNHAStatusHeartbeatProto haStatus = 2; required NNHAStatusHeartbeatProto haStatus = 2;
optional RollingUpgradeStatusProto rollingUpgradeStatus = 3;
} }
/** /**

View File

@ -133,6 +133,17 @@ message IsFormattedResponseProto {
required bool isFormatted = 1; required bool isFormatted = 1;
} }
/**
* discardSegments()
*/
message DiscardSegmentsRequestProto {
required JournalIdProto jid = 1;
required uint64 startTxId = 2;
}
message DiscardSegmentsResponseProto {
}
/** /**
* getJournalState() * getJournalState()
*/ */
@ -236,6 +247,8 @@ message AcceptRecoveryResponseProto {
service QJournalProtocolService { service QJournalProtocolService {
rpc isFormatted(IsFormattedRequestProto) returns (IsFormattedResponseProto); rpc isFormatted(IsFormattedRequestProto) returns (IsFormattedResponseProto);
rpc discardSegments(DiscardSegmentsRequestProto) returns (DiscardSegmentsResponseProto);
rpc getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto); rpc getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto);
rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto); rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto);

View File

@ -199,6 +199,10 @@ enum Status {
ERROR_ACCESS_TOKEN = 5; ERROR_ACCESS_TOKEN = 5;
CHECKSUM_OK = 6; CHECKSUM_OK = 6;
ERROR_UNSUPPORTED = 7; ERROR_UNSUPPORTED = 7;
OOB_RESTART = 8; // Quick restart
OOB_RESERVED1 = 9; // Reserved
OOB_RESERVED2 = 10; // Reserved
OOB_RESERVED3 = 11; // Reserved
} }
message PipelineAckProto { message PipelineAckProto {

View File

@ -71,6 +71,7 @@ message NameSystemSection {
optional uint64 genstampV1Limit = 4; optional uint64 genstampV1Limit = 4;
optional uint64 lastAllocatedBlockId = 5; optional uint64 lastAllocatedBlockId = 5;
optional uint64 transactionId = 6; optional uint64 transactionId = 6;
optional uint64 rollingUpgradeStartTime = 7;
} }
/** /**

View File

@ -60,6 +60,15 @@ message DatanodeIDProto {
optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
} }
/**
* Datanode local information
*/
message DatanodeLocalInfoProto {
required string softwareVersion = 1;
required string configVersion = 2;
required uint64 uptime = 3;
}
/** /**
* DatanodeInfo array * DatanodeInfo array
*/ */
@ -459,4 +468,9 @@ message SnapshotInfoProto {
// TODO: do we need access time? // TODO: do we need access time?
} }
/**
* Rolling upgrade status
*/
message RollingUpgradeStatusProto {
required string blockPoolId = 1;
}

Some files were not shown because too many files have changed in this diff Show More