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 class LightWeightGSet<K, E extends K> implements GSet<K, E> {
out.println("\n]");
}
private class SetIterator implements Iterator<E> {
public class SetIterator implements Iterator<E> {
/** The starting modification for fail-fast. */
private int iterModification = modification;
/** The current index of the entry array. */
private int index = -1;
private LinkedElement cur = null;
private LinkedElement next = nextNonemptyEntry();
private boolean trackModification = true;
/** Find the next nonempty entry starting at (index + 1). */
private LinkedElement nextNonemptyEntry() {
@ -259,7 +260,7 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
}
private void ensureNext() {
if (modification != iterModification) {
if (trackModification && modification != iterModification) {
throw new ConcurrentModificationException("modification=" + modification
+ " != iterModification = " + iterModification);
}
@ -304,6 +305,10 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
iterModification++;
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-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
INCOMPATIBLE CHANGES

View File

@ -395,7 +395,7 @@ public class BookKeeperJournalManager implements JournalManager {
try {
String znodePath = inprogressZNode(txId);
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
* This can fail if for some reason our write lock has
* expired (@see WriteLock) and another process has managed to
@ -659,6 +659,11 @@ public class BookKeeperJournalManager implements JournalManager {
}
}
@Override
public void discardSegments(long startTxId) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void close() throws IOException {
try {

View File

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

View File

@ -30,6 +30,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_
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_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_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
@ -138,11 +140,13 @@ import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
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.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -269,6 +273,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
final int getFileBlockStorageLocationsTimeoutMs;
final int retryTimesForGetLastBlockLength;
final int retryIntervalForGetLastBlockLength;
final long datanodeRestartTimeout;
final boolean useLegacyBlockReader;
final boolean useLegacyBlockReaderLocal;
@ -419,6 +424,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
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) {
@ -2476,6 +2485,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
namenode.finalizeUpgrade();
}
RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
return namenode.rollingUpgrade(action);
}
/**
*/
@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 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 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_DEFAULT = "localhost:50100";
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 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 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 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 String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
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 boolean DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT = false;
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.IOException;
import java.io.InputStream;
import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.BufferOverflowException;
@ -344,6 +345,8 @@ public class DFSOutputStream extends FSOutputSummer
private String[] favoredNodes;
volatile boolean hasError = false;
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 long bytesSent = 0; // number of bytes that've been sent
@ -479,7 +482,7 @@ public class DFSOutputStream extends FSOutputSummer
try {
// process datanode IO errors if any
boolean doSleep = false;
if (hasError && errorIndex>=0) {
if (hasError && (errorIndex >= 0 || restartingNodeIndex >= 0)) {
doSleep = processDatanodeError();
}
@ -577,8 +580,12 @@ public class DFSOutputStream extends FSOutputSummer
blockStream.flush();
} catch (IOException e) {
// HDFS-3398 treat primary DN is down since client is unable to
// write to primary DN
errorIndex = 0;
// write to primary DN. If a failed or restarting node has already
// 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;
}
lastPacket = Time.now();
@ -615,12 +622,16 @@ public class DFSOutputStream extends FSOutputSummer
Thread.sleep(artificialSlowdown);
}
} 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) {
setLastException((IOException)e);
}
hasError = true;
if (errorIndex == -1) { // not a datanode error
if (errorIndex == -1 && restartingNodeIndex == -1) {
// Not a datanode issue
streamerClosed = true;
}
}
@ -699,6 +710,65 @@ public class DFSOutputStream extends FSOutputSummer
}
}
// 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
// from the ackQueue when its response arrives.
@ -732,8 +802,20 @@ public class DFSOutputStream extends FSOutputSummer
// processes response status from datanodes.
for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; 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) {
errorIndex = i; // first bad datanode
setErrorIndex(i); // first bad datanode
throw new IOException("Bad response " + reply +
" for block " + block +
" from datanode " +
@ -782,12 +864,16 @@ public class DFSOutputStream extends FSOutputSummer
setLastException((IOException)e);
}
hasError = true;
errorIndex = errorIndex==-1 ? 0 : errorIndex;
// If no explicit error report was received, mark the primary
// node as failed.
tryMarkPrimaryDatanodeFailed();
synchronized (dataQueue) {
dataQueue.notifyAll();
}
DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
+ " for block " + block, e);
if (restartingNodeIndex == -1) {
DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
+ " for block " + block, e);
}
responderClosed = true;
}
}
@ -1006,6 +1092,24 @@ public class DFSOutputStream extends FSOutputSummer
boolean success = false;
long newGS = 0L;
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;
// remove bad datanode from list of datanodes.
// If errorIndex was not set (i.e. appends), then do not remove
@ -1042,7 +1146,24 @@ public class DFSOutputStream extends FSOutputSummer
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);
errorIndex = -1;
}
@ -1071,7 +1192,34 @@ public class DFSOutputStream extends FSOutputSummer
} else {
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) {
// update pipeline at the namenode
@ -1149,6 +1297,7 @@ public class DFSOutputStream extends FSOutputSummer
}
Status pipelineStatus = SUCCESS;
String firstBadLink = "";
boolean checkRestart = false;
if (DFSClient.LOG.isDebugEnabled()) {
for (int i = 0; i < nodes.length; i++) {
DFSClient.LOG.debug("pipeline = " + nodes[i]);
@ -1197,6 +1346,16 @@ public class DFSOutputStream extends FSOutputSummer
pipelineStatus = resp.getStatus();
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 == Status.ERROR_ACCESS_TOKEN) {
throw new InvalidBlockTokenException(
@ -1210,9 +1369,12 @@ public class DFSOutputStream extends FSOutputSummer
assert null == blockStream : "Previous blockStream unclosed";
blockStream = out;
result = true; // success
restartingNodeIndex = -1;
hasError = false;
} 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) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+ "encryption key was invalid when connecting to "
@ -1235,8 +1397,18 @@ public class DFSOutputStream extends FSOutputSummer
}
}
} else {
assert checkRestart == false;
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;
setLastException(ie);
result = false; // error

View File

@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.client.HdfsAdmin;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
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.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -68,11 +69,12 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
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.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@ -1037,6 +1039,14 @@ public class DistributedFileSystem extends FileSystem {
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
* file.

View File

@ -127,4 +127,22 @@ public interface ClientDatanodeProtocol {
*/
HdfsBlocksMetadata getHdfsBlocksMetadata(String blockPoolId,
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.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission;
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.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@ -763,6 +764,15 @@ public interface ClientProtocol {
@Idempotent
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
* 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;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
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
@ -79,6 +86,24 @@ public class HdfsConstants {
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
public static enum DatanodeReportType {
ALL, LIVE, DEAD
@ -103,13 +128,6 @@ public class HdfsConstants {
*/
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.
* <p>
@ -120,6 +138,20 @@ public class HdfsConstants {
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
*/

View File

@ -17,9 +17,10 @@
*/
package org.apache.hadoop.hdfs.protocol;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Comparator;
import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience;
@ -43,7 +44,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
*/
@InterfaceAudience.Private
public class LayoutVersion {
/**
* 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
@ -54,19 +54,27 @@ public class LayoutVersion {
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>
* 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 spacific ancestor
* its immediate predecessor, use the constructor where a specific ancestor
* can be passed.
* </li>
* </ul>
*/
public static enum Feature {
public static enum Feature implements LayoutFeature {
NAMESPACE_QUOTA(-16, "Support for namespace quotas"),
FILE_ACCESS_TIME(-17, "Support for access time on files"),
DISKSPACE_QUOTA(-18, "Support for disk space quotas"),
@ -119,12 +127,8 @@ public class LayoutVersion {
RESERVED_REL2_4_0(-54, -51, "Reserved for release 2.4.0", true,
PROTOBUF_FORMAT, EXTENDED_ACL);
final int lv;
final int ancestorLV;
final String description;
final boolean reserved;
final Feature[] specialFeatures;
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
@ -140,16 +144,35 @@ public class LayoutVersion {
* @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
* verions
* 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;
}
}
/** 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.ancestorLV = ancestorLV;
this.description = description;
this.reserved = reserved;
specialFeatures = features;
this.specialFeatures = specialFeatures;
}
/**
@ -179,80 +202,91 @@ public class LayoutVersion {
public boolean isReservedForOldRelease() {
return reserved;
}
public LayoutFeature[] getSpecialFeatures() {
return specialFeatures;
}
}
// Build layout version and corresponding feature matrix
static final Map<Integer, EnumSet<Feature>>map =
new HashMap<Integer, EnumSet<Feature>>();
// Static initialization
static {
initMap();
static class LayoutFeatureComparator implements Comparator<LayoutFeature> {
@Override
public int compare(LayoutFeature arg0, LayoutFeature arg1) {
return arg0.getInfo().getLayoutVersion()
- arg1.getInfo().getLayoutVersion();
}
}
/**
* Initialize the map of a layout version and EnumSet of {@link Feature}s
* supported.
*/
private static void initMap() {
public static void updateMap(Map<Integer, SortedSet<LayoutFeature>> map,
LayoutFeature[] features) {
// Go through all the enum constants and build a map of
// LayoutVersion <-> EnumSet of all supported features in that LayoutVersion
for (Feature f : Feature.values()) {
EnumSet<Feature> ancestorSet = map.get(f.ancestorLV);
// LayoutVersion <-> Set of all supported features in that LayoutVersion
for (LayoutFeature f : features) {
final FeatureInfo info = f.getInfo();
SortedSet<LayoutFeature> ancestorSet = map.get(info.getAncestorLayoutVersion());
if (ancestorSet == null) {
ancestorSet = EnumSet.noneOf(Feature.class); // Empty enum set
map.put(f.ancestorLV, ancestorSet);
// Empty set
ancestorSet = new TreeSet<LayoutFeature>(new LayoutFeatureComparator());
map.put(info.getAncestorLayoutVersion(), ancestorSet);
}
EnumSet<Feature> featureSet = EnumSet.copyOf(ancestorSet);
if (f.specialFeatures != null) {
for (Feature specialFeature : f.specialFeatures) {
SortedSet<LayoutFeature> featureSet = new TreeSet<LayoutFeature>(ancestorSet);
if (info.getSpecialFeatures() != null) {
for (LayoutFeature specialFeature : info.getSpecialFeatures()) {
featureSet.add(specialFeature);
}
}
featureSet.add(f);
map.put(f.lv, featureSet);
map.put(info.getLayoutVersion(), featureSet);
}
}
/**
* 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();
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 ")
.append(f.lv).append(" (").
append(f.description).append(")\n");
.append(info.getLayoutVersion()).append(" (")
.append(info.getDescription()).append(")\n");
}
buf.append("\n\nLayoutVersion and supported features:\n");
for (Feature f : Feature.values()) {
buf.append(f.lv).append(": ").append(map.get(f.lv))
.append("\n");
for (LayoutFeature f : values) {
final FeatureInfo info = f.getInfo();
buf.append(info.getLayoutVersion()).append(": ")
.append(map.get(info.getLayoutVersion())).append("\n");
}
return buf.toString();
}
/**
* Returns true if a given feature is supported in the given layout version
* @param map layout feature map
* @param f Feature
* @param lv LayoutVersion
* @return true if {@code f} is supported in layout version {@code lv}
*/
public static boolean supports(final Feature f, final int lv) {
final EnumSet<Feature> set = map.get(lv);
public static boolean supports(Map<Integer, SortedSet<LayoutFeature>> map,
final LayoutFeature f, final int lv) {
final SortedSet<LayoutFeature> set = map.get(lv);
return set != null && set.contains(f);
}
/**
* Get the current layout version
*/
public static int getCurrentLayoutVersion() {
Feature[] values = Feature.values();
for (int i = values.length -1; i >= 0; i--) {
if (!values[i].isReservedForOldRelease()) {
return values[i].lv;
public static int getCurrentLayoutVersion(LayoutFeature[] features) {
return getLastNonReservedFeature(features).getInfo().getLayoutVersion();
}
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.");

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 java.util.Arrays;
import org.apache.hadoop.classification.InterfaceAudience;
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.PipelineAckProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import com.google.protobuf.TextFormat;
/** Pipeline Acknowledgment **/
@ -38,6 +40,21 @@ import com.google.protobuf.TextFormat;
public class PipelineAck {
PipelineAckProto proto;
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 **/
public PipelineAck() {
@ -103,14 +120,57 @@ public class PipelineAck {
* @return true if all statuses are SUCCESS
*/
public boolean isSuccess() {
for (DataTransferProtos.Status reply : proto.getStatusList()) {
if (reply != DataTransferProtos.Status.SUCCESS) {
for (Status reply : proto.getStatusList()) {
if (reply != Status.SUCCESS) {
return false;
}
}
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 ****/
public void readFields(InputStream in) throws IOException {
proto = PipelineAckProto.parseFrom(vintPrefixed(in));

View File

@ -30,6 +30,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Delete
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.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.GetHdfsBlockLocationsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto.Builder;
@ -37,10 +39,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetRep
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.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.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.VersionInfo;
import com.google.common.primitives.Longs;
import com.google.protobuf.ByteString;
@ -59,6 +64,8 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
RefreshNamenodesResponseProto.newBuilder().build();
private final static DeleteBlockPoolResponseProto DELETE_BLOCKPOOL_RESP =
DeleteBlockPoolResponseProto.newBuilder().build();
private final static ShutdownDatanodeResponseProto SHUTDOWN_DATANODE_RESP =
ShutdownDatanodeResponseProto.newBuilder().build();
private final ClientDatanodeProtocol impl;
@ -151,4 +158,28 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
builder.addAllVolumeIndexes(resp.getVolumeIndexes());
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.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
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.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
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.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.GetHdfsBlockLocationsResponseProto;
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.ShutdownDatanodeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.ipc.ProtobufHelper;
@ -81,6 +85,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
private final ClientDatanodeProtocolPB rpcProxy;
private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES =
RefreshNamenodesRequestProto.newBuilder().build();
private final static GetDatanodeInfoRequestProto VOID_GET_DATANODE_INFO =
GetDatanodeInfoRequestProto.newBuilder().build();
public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
@ -253,4 +259,27 @@ public class ClientDatanodeProtocolTranslatorPB implements
return new HdfsBlocksMetadata(blockPoolId, blockIds,
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 java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.Options.Rename;
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.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@ -34,7 +35,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
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.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@ -53,10 +54,10 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Abando
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.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.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.AllowSnapshotResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
@ -112,28 +113,28 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
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.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.ListCachePoolsResponseProto;
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.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.MetaSaveResponseProto;
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.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.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.RecoverLeaseResponseProto;
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.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.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.Rename2ResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
@ -148,6 +149,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Restor
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.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.SaveNamespaceResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
@ -711,6 +714,22 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
}
}
@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
public ListCorruptFileBlocksResponseProto listCorruptFileBlocks(
RpcController controller, ListCorruptFileBlocksRequestProto req)

View File

@ -40,8 +40,9 @@ import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
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.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@ -50,12 +51,13 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
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.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@ -66,8 +68,8 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestPr
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.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.AddCachePoolRequestProto;
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.AppendResponseProto;
@ -106,19 +108,19 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
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.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.ListCachePoolsResponseProto;
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.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.ModifyCachePoolRequestProto;
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.RemoveCachePoolRequestProto;
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.RenameRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
@ -127,6 +129,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Report
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.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.SetBalancerBandwidthRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
@ -644,6 +648,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
}
@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
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException {

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
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.RollingUpgradeStatus;
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.BlockReportResponseProto;
@ -184,7 +185,12 @@ public class DatanodeProtocolClientSideTranslatorPB implements
cmds[index] = PBHelper.convert(p);
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

View File

@ -23,6 +23,7 @@ import java.util.List;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
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.BlockReceivedAndDeletedResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
@ -121,6 +122,12 @@ public class DatanodeProtocolServerSideTranslatorPB implements
}
}
builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState()));
RollingUpgradeStatus rollingUpdateStatus = response
.getRollingUpdateStatus();
if (rollingUpdateStatus != null) {
builder.setRollingUpgradeStatus(PBHelper
.convertRollingUpgradeStatus(rollingUpdateStatus));
}
return builder.build();
}

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
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.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@ -54,17 +55,20 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
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.DatanodeLocalInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
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.DiffType;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -77,6 +81,7 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto
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.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.CacheFlagProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
@ -85,6 +90,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheP
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.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.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
@ -117,6 +124,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
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.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.ExportedBlockKeysProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
@ -135,6 +143,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
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.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.SnapshotDiffReportProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
@ -149,6 +158,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
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.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
@ -254,9 +264,9 @@ public class PBHelper {
.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(),
info.getClusterID(), info.getCTime());
info.getClusterID(), info.getCTime(), type);
}
public static NamenodeRegistrationProto convert(NamenodeRegistration reg) {
@ -267,8 +277,9 @@ public class PBHelper {
}
public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
StorageInfo si = convert(reg.getStorageInfo(), NodeType.NAME_NODE);
return new NamenodeRegistration(reg.getRpcAddress(), reg.getHttpAddress(),
convert(reg.getStorageInfo()), convert(reg.getRole()));
si, convert(reg.getRole()));
}
// DatanodeId
@ -400,9 +411,9 @@ public class PBHelper {
}
public static CheckpointSignature convert(CheckpointSignatureProto s) {
return new CheckpointSignature(PBHelper.convert(s.getStorageInfo()),
s.getBlockPoolId(), s.getMostRecentCheckpointTxId(),
s.getCurSegmentTxId());
StorageInfo si = PBHelper.convert(s.getStorageInfo(), NodeType.NAME_NODE);
return new CheckpointSignature(si, s.getBlockPoolId(),
s.getMostRecentCheckpointTxId(), s.getCurSegmentTxId());
}
public static RemoteEditLogProto convert(RemoteEditLog log) {
@ -755,9 +766,9 @@ public class PBHelper {
}
public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
StorageInfo si = convert(proto.getStorageInfo(), NodeType.DATA_NODE);
return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
.getKeys()), proto.getSoftwareVersion());
si, PBHelper.convert(proto.getKeys()), proto.getSoftwareVersion());
}
public static DatanodeCommand convert(DatanodeCommandProto proto) {
@ -1448,6 +1459,59 @@ public class PBHelper {
}
}
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) {
if (c == null)
return null;
@ -1903,6 +1967,19 @@ public class PBHelper {
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)
throws IOException {
final int firstByte = input.read();

View File

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

View File

@ -308,4 +308,13 @@ class AsyncLoggerSet {
}
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 class IPCLoggerChannel implements AsyncLogger {
});
}
@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
public String toString() {
return InetAddresses.toAddrString(addr.getAddress()) + ':' +

View File

@ -79,6 +79,7 @@ public class QuorumJournalManager implements JournalManager {
// configurable.
private static final int FORMAT_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 URI uri;
@ -492,4 +493,22 @@ public class QuorumJournalManager implements JournalManager {
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.protocol.QJournalProtocolProtos.SegmentSt
import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
import org.apache.hadoop.hdfs.server.namenode.JournalManager;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.security.KerberosInfo;
/**
@ -143,4 +144,12 @@ public interface QJournalProtocol {
*/
public void acceptRecovery(RequestInfo reqInfo,
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;
import java.io.IOException;
import java.net.URL;
import org.apache.hadoop.classification.InterfaceAudience;
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.qjournal.protocol.QJournalProtocol;
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.FinalizeLogSegmentResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
@ -39,8 +47,6 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRe
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.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.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
@ -48,13 +54,11 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogs
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.RequestInfo;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.net.URL;
/**
* Implementation for protobuf service that forwards requests
* received on {@link JournalProtocolPB} to the
@ -244,4 +248,16 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
reqInfo.hasCommittedTxId() ?
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 java.net.URL;
import org.apache.hadoop.classification.InterfaceAudience;
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.qjournal.protocol.QJournalProtocol;
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.FormatRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
@ -39,7 +42,6 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalId
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.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.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
@ -277,4 +279,16 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
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 @@ class Journal implements Closeable {
}
}
}
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 class JournalNode implements Tool, Configurable, JournalNodeMXBean {
StringUtils.startupShutdownMessage(JournalNode.class, args, LOG);
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 @@ class JournalNodeRpcServer implements QJournalProtocol {
.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 class BlockManager {
*/
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 */
private BlockPlacementPolicy blockplacement;
@ -310,6 +326,9 @@ public class BlockManager {
this.maxNumBlocksToLog =
conf.getLong(DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
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("maxReplication = " + maxReplication);
@ -2364,45 +2383,127 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
*/
public void processMisReplicatedBlocks() {
assert namesystem.hasWriteLock();
long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0, nrPostponed = 0,
nrUnderConstruction = 0;
stopReplicationInitializer();
neededReplications.clear();
for (BlockInfo block : blocksMap.getBlocks()) {
MisReplicationResult res = processMisReplicatedBlock(block);
if (LOG.isTraceEnabled()) {
LOG.trace("block " + block + ": " + res);
replicationQueuesInitializer = new Daemon() {
@Override
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:
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);
};
replicationQueuesInitializer.setName("Replication Queue Initializer");
replicationQueuesInitializer.start();
}
/*
* Stop the ongoing initialisation of replication queues
*/
private void stopReplicationInitializer() {
if (replicationQueuesInitializer != null) {
replicationQueuesInitializer.interrupt();
try {
replicationQueuesInitializer.join();
} catch (final InterruptedException e) {
LOG.warn("Interrupted while waiting for replicationQueueInitializer. Returning..");
return;
} finally {
replicationQueuesInitializer = null;
}
}
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);
}
/*
* Since the BlocksMapGset does not throw the ConcurrentModificationException
* and supports further iteration after modification to list, there is a
* chance of missing the newly added block while iterating. Since every
* 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 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
}
public void shutdown() {
stopReplicationInitializer();
blocksMap.close();
}
}

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet;
import org.apache.hadoop.util.LightWeightGSet.SetIterator;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
@ -66,7 +67,20 @@ class BlocksMap {
BlocksMap(int capacity) {
// Use 2% of total memory to size the GSet 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 @@ package org.apache.hadoop.hdfs.server.common;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext;
import com.google.common.base.Preconditions;
/************************************
* Some handy internal HDFS constants
*
@ -43,6 +47,33 @@ public final class HdfsServerConstants {
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 */
static public enum StartupOption{
FORMAT ("-format"),
@ -54,6 +85,7 @@ public final class HdfsServerConstants {
UPGRADE ("-upgrade"),
ROLLBACK("-rollback"),
FINALIZE("-finalize"),
ROLLINGUPGRADE("-rollingUpgrade"),
IMPORT ("-importCheckpoint"),
BOOTSTRAPSTANDBY("-bootstrapStandby"),
INITIALIZESHAREDEDITS("-initializeSharedEdits"),
@ -61,12 +93,18 @@ public final class HdfsServerConstants {
FORCE("-force"),
NONINTERACTIVE("-nonInteractive"),
RENAMERESERVED("-renameReserved");
private static final Pattern ENUM_WITH_ROLLING_UPGRADE_OPTION = Pattern.compile(
"(\\w+)\\((\\w+)\\)");
private final String name;
// Used only with format and upgrade options
private String clusterId = null;
// Used only by rolling upgrade
private RollingUpgradeStartupOption rollingUpgradeStartupOption;
// Used only with format option
private boolean isForceFormat = false;
private boolean isInteractiveFormat = true;
@ -94,6 +132,16 @@ public final class HdfsServerConstants {
public String getClusterId() {
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() {
if (!name.equals(RECOVER.name))
@ -124,6 +172,27 @@ public final class HdfsServerConstants {
public void setInteractiveFormat(boolean 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

View File

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

View File

@ -26,26 +26,22 @@ import java.lang.management.ManagementFactory;
import java.nio.channels.FileLock;
import java.nio.channels.OverlappingFileLockException;
import java.util.ArrayList;
import java.util.List;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
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.StartupOption;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.util.VersionInfo;
import com.google.common.base.Preconditions;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
@ -121,7 +117,6 @@ public abstract class Storage extends StorageInfo {
public boolean isOfType(StorageDirType type);
}
protected NodeType storageType; // Type of the node using this storage
protected List<StorageDirectory> storageDirs = new ArrayList<StorageDirectory>();
private class DirIterator implements Iterator<StorageDirectory> {
@ -752,13 +747,11 @@ public abstract class Storage extends StorageInfo {
* Create empty storage info of the specified type
*/
protected Storage(NodeType type) {
super();
this.storageType = type;
super(type);
}
protected Storage(NodeType type, StorageInfo storageInfo) {
protected Storage(StorageInfo storageInfo) {
super(storageInfo);
this.storageType = type;
}
public int getNumStorageDirs() {
@ -888,22 +881,6 @@ public abstract class Storage extends StorageInfo {
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.
* Should be overloaded if additional fields need to be set.
@ -917,7 +894,7 @@ public abstract class Storage extends StorageInfo {
props.setProperty("storageType", storageType.toString());
props.setProperty("namespaceID", String.valueOf(namespaceID));
// Set clusterID in version with federation support
if (versionSupportsFederation()) {
if (versionSupportsFederation(getServiceLayoutFeatureMap())) {
props.setProperty("clusterID", clusterID);
}
props.setProperty("cTime", String.valueOf(cTime));
@ -1016,7 +993,7 @@ public abstract class Storage extends StorageInfo {
* @throws IOException
*/
public void writeAll() throws IOException {
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.layoutVersion = getServiceLayoutVersion();
for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
writeProperties(it.next());
}
@ -1039,73 +1016,9 @@ public abstract class Storage extends StorageInfo {
public static String getRegistrationID(StorageInfo storage) {
return "NS-" + Integer.toString(storage.getNamespaceID())
+ "-" + storage.getClusterID()
+ "-" + Integer.toString(storage.getLayoutVersion())
+ "-" + 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) {
for (int lv203 : LAYOUT_VERSIONS_203) {
if (lv203 == layoutVersion) {

View File

@ -17,9 +17,20 @@
*/
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.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
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;
@ -34,20 +45,26 @@ public class StorageInfo {
public int namespaceID; // id of the file system
public String clusterID; // id of the cluster
public long cTime; // creation time of the file system state
public StorageInfo () {
this(0, 0, "", 0L);
protected final NodeType storageType; // Type of the node using this storage
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;
clusterID = cid;
namespaceID = nsID;
cTime = cT;
storageType = type;
}
public StorageInfo(StorageInfo from) {
setStorageInfo(from);
this(from.layoutVersion, from.namespaceID, from.clusterID, from.cTime,
from.storageType);
}
/**
@ -80,8 +97,10 @@ public class StorageInfo {
cTime = from.cTime;
}
public boolean versionSupportsFederation() {
return LayoutVersion.supports(Feature.FEDERATION, layoutVersion);
public boolean versionSupportsFederation(
Map<Integer, SortedSet<LayoutFeature>> map) {
return LayoutVersion.supports(map, LayoutVersion.Feature.FEDERATION,
layoutVersion);
}
@Override
@ -96,4 +115,97 @@ public class StorageInfo {
return Joiner.on(":").join(
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;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@ -32,24 +29,15 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
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.*;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
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
@ -414,6 +402,18 @@ class BPOfferService {
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
* response from one of the actors.
@ -612,7 +612,8 @@ class BPOfferService {
// See HDFS-2987.
throw new UnsupportedOperationException("Received unimplemented DNA_SHUTDOWN");
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) :
"BP " + getBlockPoolId() + " received DNA_FINALIZE " +
"for other block pool " + bp;

View File

@ -33,8 +33,8 @@ import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
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.RollingUpgradeStatus;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
@ -204,14 +204,6 @@ class BPServiceActor implements Runnable {
"DataNode version '" + dnVersion + "' but is within acceptable " +
"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 {
@ -625,6 +617,20 @@ class BPServiceActor implements Runnable {
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,
* forever calling remote NameNode functions.
@ -671,6 +677,10 @@ class BPServiceActor implements Runnable {
this, resp.getNameNodeHaState());
state = resp.getNameNodeHaState().getState();
if (state == HAServiceState.ACTIVE) {
handleRollingUpgradeStatus(resp);
}
long startProcessCommands = now();
if (!processCommand(resp.getCommands()))
continue;

View File

@ -18,6 +18,20 @@
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.IOException;
import java.util.ArrayList;
@ -27,20 +41,6 @@ import java.util.Properties;
import java.util.regex.Matcher;
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
* block pool id, on this DataNode.
@ -58,14 +58,24 @@ import org.apache.hadoop.util.Daemon;
*/
@InterfaceAudience.Private
public class BlockPoolSliceStorage extends Storage {
private static final Pattern BLOCK_POOL_PATH_PATTERN = Pattern
.compile("^(.*)"
+ "(\\/BP-[0-9]+\\-\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\-[0-9]+\\/.*)$");
static final String TRASH_ROOT_DIR = "trash";
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
public BlockPoolSliceStorage(StorageInfo storageInfo, String bpid) {
super(NodeType.DATA_NODE, storageInfo);
super(storageInfo);
blockpoolID = bpid;
}
@ -93,9 +103,7 @@ public class BlockPoolSliceStorage extends Storage {
*/
void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
Collection<File> dataDirs, StartupOption startOpt) throws IOException {
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion()
: "Block-pool and name-node layout versions must be the same.";
LOG.info("Analyzing storage directories for bpid " + nsInfo.getBlockPoolID());
// 1. For each BP data directory analyze the state and
// check whether all is consistent before transitioning.
this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
@ -143,8 +151,6 @@ public class BlockPoolSliceStorage extends Storage {
// while others could be up-to-date for the regular startup.
for (int idx = 0; idx < getNumStorageDirs(); idx++) {
doTransition(getStorageDir(idx), nsInfo, startOpt);
assert getLayoutVersion() == nsInfo.getLayoutVersion()
: "Data-node and name-node layout versions must be the same.";
assert getCTime() == nsInfo.getCTime()
: "Data-node and name-node CTimes must be the same.";
}
@ -167,7 +173,7 @@ public class BlockPoolSliceStorage extends 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
* @throws IOException Signals that an I/O exception has occurred.
*/
@ -175,11 +181,10 @@ public class BlockPoolSliceStorage extends Storage {
LOG.info("Formatting block pool " + blockpoolID + " directory "
+ bpSdir.getCurrentDir());
bpSdir.clearDirectory(); // create directory
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
this.cTime = nsInfo.getCTime();
this.namespaceID = nsInfo.getNamespaceID();
this.blockpoolID = nsInfo.getBlockPoolID();
this.storageType = NodeType.DATA_NODE;
writeProperties(bpSdir);
}
@ -206,7 +211,7 @@ public class BlockPoolSliceStorage extends Storage {
if (!blockpoolID.equals("") && !blockpoolID.equals(bpid)) {
throw new InconsistentFSStateException(storage,
"Unexepcted blockpoolID " + bpid + " . Expected " + blockpoolID);
"Unexpected blockpoolID " + bpid + ". Expected " + blockpoolID);
}
blockpoolID = bpid;
}
@ -230,7 +235,6 @@ public class BlockPoolSliceStorage extends Storage {
* Upgrade if this.LV > LAYOUT_VERSION || this.cTime < namenode.cTime Regular
* 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 nsInfo namespace info
* @param startOpt startup option
@ -238,12 +242,18 @@ public class BlockPoolSliceStorage extends Storage {
*/
private void doTransition(StorageDirectory sd,
NamespaceInfo nsInfo, StartupOption startOpt) throws IOException {
if (startOpt == StartupOption.ROLLBACK)
if (startOpt == StartupOption.ROLLBACK) {
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);
checkVersionUpgradable(this.layoutVersion);
assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION
assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION
: "Future version is not allowed";
if (getNamespaceID() != nsInfo.getNamespaceID()) {
throw new IOException("Incompatible namespaceIDs in "
@ -257,11 +267,11 @@ public class BlockPoolSliceStorage extends Storage {
+ nsInfo.getBlockPoolID() + "; datanode blockpoolID = "
+ blockpoolID);
}
if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION
if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION
&& this.cTime == nsInfo.getCTime()) {
return; // regular startup
}
if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION
if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION
|| this.cTime < nsInfo.getCTime()) {
doUpgrade(sd, nsInfo); // upgrade
return;
@ -294,7 +304,8 @@ public class BlockPoolSliceStorage extends Storage {
*/
void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException {
// Upgrading is applicable only to release with federation or after
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
return;
}
LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
@ -322,18 +333,20 @@ public class BlockPoolSliceStorage extends Storage {
File bpTmpDir = bpSd.getPreviousTmp();
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);
// 3. Create new <SD>/current with block files hardlinks and VERSION
linkAllBlocks(bpTmpDir, bpCurDir);
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
assert this.namespaceID == nsInfo.getNamespaceID()
: "Data-node and name-node layout versions must be the same.";
this.cTime = nsInfo.getCTime();
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);
LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot()
+ " is complete");
@ -349,7 +362,8 @@ public class BlockPoolSliceStorage extends Storage {
* @throws IOException if the directory is not empty or it can not be removed
*/
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()) {
if (FileUtil.list(detachDir).length != 0) {
@ -363,6 +377,43 @@ public class BlockPoolSliceStorage extends Storage {
}
}
/**
* 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
* If previous directory exists:
@ -389,13 +440,13 @@ public class BlockPoolSliceStorage extends Storage {
// the namespace state or can be further upgraded to it.
// In another word, we can only roll back when ( storedLV >= software LV)
// && ( DN.previousCTime <= NN.ctime)
if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION &&
if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION &&
prevInfo.getCTime() <= nsInfo.getCTime())) { // cannot rollback
throw new InconsistentFSStateException(bpSd.getRoot(),
"Cannot rollback to a newer state.\nDatanode previous state: LV = "
+ prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
+ " 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()
@ -478,9 +529,6 @@ public class BlockPoolSliceStorage extends Storage {
/**
* gets the data node storage directory based on block pool storage
*
* @param bpRoot
* @return
*/
private static String getDataNodeStorageRoot(String bpRoot) {
Matcher matcher = BLOCK_POOL_PATH_PATTERN.matcher(bpRoot);
@ -510,4 +558,66 @@ public class BlockPoolSliceStorage extends Storage {
public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
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.BufferedOutputStream;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileDescriptor;
import java.io.FileOutputStream;
import java.io.FileWriter;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
@ -45,6 +47,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
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.ReplicaOutputStreams;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
@ -52,6 +55,7 @@ import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
@ -116,6 +120,7 @@ class BlockReceiver implements Closeable {
private final boolean isTransfer;
private boolean syncOnClose;
private long restartBudget;
BlockReceiver(final ExtendedBlock block, final DataInputStream in,
final String inAddr, final String myAddr,
@ -135,6 +140,7 @@ class BlockReceiver implements Closeable {
this.clientname = clientname;
this.isDatanode = clientname.length() == 0;
this.isClient = !this.isDatanode;
this.restartBudget = datanode.getDnConf().restartReplicaExpiry;
//for datanode, we have
//1: clientName.length() == 0, and
@ -723,19 +729,71 @@ class BlockReceiver implements Closeable {
}
} catch (IOException ioe) {
LOG.info("Exception for " + block, ioe);
throw ioe;
if (datanode.isRestarting()) {
// 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 {
if (!responderClosed) { // Abnormal termination of the flow above
IOUtils.closeStream(this);
// Clear the previous interrupt state of this thread.
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) {
// 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();
}
IOUtils.closeStream(this);
cleanupBlock();
}
if (responder != null) {
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()) {
String msg = "Join on responder thread " + responder
+ " timed out";
@ -744,7 +802,10 @@ class BlockReceiver implements Closeable {
}
} catch (InterruptedException e) {
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;
}
@ -862,6 +923,7 @@ class BlockReceiver implements Closeable {
private final PacketResponderType type;
/** for log and error messages */
private final String myString;
private boolean sending = false;
@Override
public String toString() {
@ -887,7 +949,9 @@ class BlockReceiver implements Closeable {
}
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 @@ class BlockReceiver implements Closeable {
if(LOG.isDebugEnabled()) {
LOG.debug(myString + ": enqueue " + p);
}
synchronized(this) {
synchronized(ackQueue) {
if (running) {
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 */
synchronized Packet waitForAckHead(long seqno) throws InterruptedException {
while (isRunning() && ackQueue.size() == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug(myString + ": seqno=" + seqno +
" waiting for local datanode to finish write.");
Packet waitForAckHead(long seqno) throws InterruptedException {
synchronized(ackQueue) {
while (isRunning() && ackQueue.size() == 0) {
if (LOG.isDebugEnabled()) {
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.
*/
@Override
public synchronized void close() {
while (isRunning() && ackQueue.size() != 0) {
try {
wait();
} catch (InterruptedException e) {
running = false;
Thread.currentThread().interrupt();
public void close() {
synchronized(ackQueue) {
while (isRunning() && ackQueue.size() != 0) {
try {
ackQueue.wait();
} catch (InterruptedException e) {
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 @@ class BlockReceiver implements Closeable {
if (LOG.isDebugEnabled()) {
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();
}
if (seqno != PipelineAck.UNKOWN_SEQNO
@ -1025,6 +1150,9 @@ class BlockReceiver implements Closeable {
* status back to the client because this datanode has a problem.
* The upstream datanode will detect that this datanode is bad, and
* rightly so.
*
* The receiver thread can also interrupt this thread for sending
* an out-of-band response upstream.
*/
LOG.info(myString + ": Thread is interrupted.");
running = false;
@ -1094,17 +1222,64 @@ class BlockReceiver implements Closeable {
}
/**
* The wrapper for the unprotected version. This is only called by
* the responder's run() method.
*
* @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 sendAckUpstream(PipelineAck ack, long seqno,
long totalAckTimeNanos, long offsetInBlock,
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;
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;
} else {
short ackLen = type == PacketResponderType.LAST_IN_PIPELINE ? 0 : ack
@ -1130,7 +1305,6 @@ class BlockReceiver implements Closeable {
&& offsetInBlock > replicaInfo.getBytesAcked()) {
replicaInfo.setBytesAcked(offsetInBlock);
}
// send my ack back to upstream datanode
replyAck.write(upstreamOut);
upstreamOut.flush();
@ -1152,9 +1326,11 @@ class BlockReceiver implements Closeable {
*
* This should be called only when the ack queue is not empty
*/
private synchronized void removeAckHead() {
ackQueue.removeFirst();
notifyAll();
private void removeAckHead() {
synchronized(ackQueue) {
ackQueue.removeFirst();
ackQueue.notifyAll();
}
}
}

View File

@ -46,6 +46,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NA
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_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.hdfs.DFSConfigKeys;
@ -82,6 +84,7 @@ public class DNConf {
final String encryptionAlgorithm;
final long xceiverStopTimeout;
final long restartReplicaExpiry;
final long maxLockedMemory;
@ -157,6 +160,10 @@ public class DNConf {
this.maxLockedMemory = conf.getLong(
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
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.

View File

@ -17,41 +17,10 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
import java.io.BufferedOutputStream;
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 com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -69,37 +38,15 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.net.DomainPeerServer;
import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
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.*;
import org.apache.hadoop.hdfs.protocol.datatransfer.*;
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.Status;
import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
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.protocolPB.*;
import org.apache.hadoop.hdfs.security.token.block.*;
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.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@ -113,11 +60,7 @@ import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMetho
import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
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.server.protocol.*;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.http.HttpConfig;
@ -140,21 +83,24 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.*;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
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 com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingService;
import javax.management.ObjectName;
import java.io.*;
import java.lang.management.ManagementFactory;
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
@ -211,7 +157,14 @@ public class DataNode extends Configured
static final Log ClientTraceLog =
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;
/**
@ -223,6 +176,8 @@ public class DataNode extends Configured
}
volatile boolean shouldRun = true;
volatile boolean shutdownForUpgrade = false;
private boolean shutdownInProgress = false;
private BlockPoolManager blockPoolManager;
volatile FsDatasetSpi<? extends FsVolumeSpi> data = null;
private String clusterId = null;
@ -266,6 +221,7 @@ public class DataNode extends Configured
private SecureResources secureResources = null;
private List<StorageLocation> dataDirs;
private Configuration conf;
private String confVersion;
private final long maxNumberOfBlocksToLog;
private final List<String> usersWithLocalPathAccess;
@ -294,6 +250,11 @@ public class DataNode extends Configured
DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
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.
if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT)) {
@ -1235,9 +1196,31 @@ public class DataNode extends Configured
// offerServices may be modified.
BPOfferService[] bposArray = this.blockPoolManager == null ? null
: 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();
// Stop the web server
if (infoServer != null) {
try {
infoServer.stop();
@ -1245,26 +1228,24 @@ public class DataNode extends Configured
LOG.warn("Exception shutting down DataNode", e);
}
}
if (ipcServer != null) {
ipcServer.stop();
}
if (pauseMonitor != null) {
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
if (this.threadGroup != null) {
int sleepMs = 2;
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 " +
this.threadGroup.activeCount());
if (this.threadGroup.activeCount() == 0) {
@ -1294,7 +1275,13 @@ public class DataNode extends Configured
} 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) {
try {
this.blockPoolManager.shutDownAll(bposArray);
@ -1321,6 +1308,13 @@ public class DataNode extends Configured
dataNodeInfoBeanName = null;
}
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 class DataNode extends Configured
/** Instantiate & Start a single datanode daemon and wait for it to finish.
* If this thread is specifically interrupted, it will stop waiting.
*/
@VisibleForTesting
public static DataNode createDataNode(String args[],
Configuration conf) throws IOException {
return createDataNode(args, conf, null);
@ -1801,6 +1796,7 @@ public class DataNode extends Configured
/** Instantiate & Start a single datanode daemon and wait for it to finish.
* If this thread is specifically interrupted, it will stop waiting.
*/
@VisibleForTesting
@InterfaceAudience.Private
public static DataNode createDataNode(String args[], Configuration conf,
SecureResources resources) throws IOException {
@ -1819,7 +1815,11 @@ public class DataNode extends Configured
&& blockPoolManager.getAllNamenodeThreads().length == 0) {
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) {
LOG.warn("Received exception in Datanode#join: " + ex);
}
@ -1911,25 +1911,28 @@ public class DataNode extends Configured
*
* @return false if passed argements are incorrect
*/
private static boolean parseArguments(String args[],
Configuration conf) {
int argsLen = (args == null) ? 0 : args.length;
@VisibleForTesting
static boolean parseArguments(String args[], Configuration conf) {
StartupOption startOpt = StartupOption.REGULAR;
for(int i=0; i < argsLen; i++) {
String cmd = args[i];
int i = 0;
if (args != null && args.length != 0) {
String cmd = args[i++];
if ("-r".equalsIgnoreCase(cmd) || "--rack".equalsIgnoreCase(cmd)) {
LOG.error("-r, --rack arguments are not supported anymore. RackID " +
"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;
} 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);
return true;
return (args == null || i == args.length); // Fail if more than one cmd specified!
}
private static void setStartupOption(Configuration conf, StartupOption opt) {
@ -1937,8 +1940,9 @@ public class DataNode extends Configured
}
static StartupOption getStartupOption(Configuration conf) {
return StartupOption.valueOf(conf.get(DFS_DATANODE_STARTUP_KEY,
StartupOption.REGULAR.toString()));
String value = conf.get(DFS_DATANODE_STARTUP_KEY,
StartupOption.REGULAR.toString());
return StartupOption.getEnum(value);
}
/**
@ -1969,11 +1973,15 @@ public class DataNode extends Configured
public static void secureMain(String args[], SecureResources resources) {
int errorCode = 0;
try {
StringUtils.startupShutdownMessage(DataNode.class, args, LOG);
DataNode datanode = createDataNode(args, null, resources);
if (datanode != null)
if (datanode != null) {
datanode.join();
} else {
errorCode = 1;
}
} catch (Throwable e) {
LOG.fatal("Exception in secureMain", e);
terminate(1, e);
@ -1983,7 +1991,7 @@ public class DataNode extends Configured
// condition was not met. Also, In secure mode, control will go to Jsvc
// and Datanode process hangs if it does not exit.
LOG.warn("Exiting Datanode");
terminate(0);
terminate(errorCode);
}
}
@ -2447,6 +2455,43 @@ public class DataNode extends Configured
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
* @return true if the datanode is connected to a NameNode at the
@ -2472,6 +2517,10 @@ public class DataNode extends Configured
return bp != null ? bp.isAlive() : false;
}
boolean isRestarting() {
return shutdownForUpgrade;
}
/**
* A datanode is considered to be fully started if all the BP threads are
* alive and all the block pools are initialized.
@ -2520,6 +2569,11 @@ public class DataNode extends Configured
return shouldRun;
}
@VisibleForTesting
DataStorage getStorage() {
return storage;
}
public ShortCircuitRegistry getShortCircuitRegistry() {
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;
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.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
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.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@ -50,6 +38,11 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
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.
* <p>
@ -66,6 +59,13 @@ public class DataStorage extends Storage {
public final static String STORAGE_DIR_FINALIZED = "finalized";
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
* is the same as the legacy StorageID for datanodes that were
@ -84,14 +84,16 @@ public class DataStorage extends Storage {
DataStorage() {
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);
}
public DataStorage(StorageInfo storageInfo) {
super(NodeType.DATA_NODE, storageInfo);
super(storageInfo);
}
public synchronized String getDatanodeUuid() {
@ -108,6 +110,43 @@ public class DataStorage extends Storage {
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.
@ -131,10 +170,8 @@ public class DataStorage extends Storage {
// DN storage has been initialized, no need to do anything
return;
}
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
"Data-node version " + HdfsConstants.LAYOUT_VERSION +
" and name-node layout version " + nsInfo.getLayoutVersion() +
" must be the same.";
LOG.info("Data-node version: " + HdfsConstants.DATANODE_LAYOUT_VERSION
+ " and name-node layout version: " + nsInfo.getLayoutVersion());
// 1. For each data directory calculate its state and
// check whether all is consistent before transitioning.
@ -186,15 +223,13 @@ public class DataStorage extends Storage {
// while others could be uptodate for the regular startup.
for(int idx = 0; idx < getNumStorageDirs(); idx++) {
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));
}
// 3. Update all storages. Some of them might have just been formatted.
this.writeAll();
// 4. mark DN storage is initilized
// 4. mark DN storage is initialized
this.initialized = true;
}
@ -261,7 +296,7 @@ public class DataStorage extends Storage {
void format(StorageDirectory sd, NamespaceInfo nsInfo,
String datanodeUuid) throws IOException {
sd.clearDirectory(); // create directory
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
this.clusterID = nsInfo.getClusterID();
this.namespaceID = nsInfo.getNamespaceID();
this.cTime = 0;
@ -297,7 +332,8 @@ public class DataStorage extends Storage {
}
// 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));
}
}
@ -321,11 +357,12 @@ public class DataStorage extends Storage {
setLayoutVersion(props, sd);
}
setcTime(props, sd);
setStorageType(props, sd);
checkStorageType(props, sd);
setClusterId(props, layoutVersion, sd);
// Read NamespaceID in version before federation
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
if (!DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion)) {
setNamespaceID(props, sd);
}
@ -414,11 +451,12 @@ public class DataStorage extends Storage {
}
readProperties(sd);
checkVersionUpgradable(this.layoutVersion);
assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION :
assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION :
"Future version is not allowed";
boolean federationSupported =
LayoutVersion.supports(Feature.FEDERATION, layoutVersion);
DataNodeLayoutVersion.supports(
LayoutVersion.Feature.FEDERATION, layoutVersion);
// For pre-federation version - validate the namespaceID
if (!federationSupported &&
getNamespaceID() != nsInfo.getNamespaceID()) {
@ -440,24 +478,22 @@ public class DataStorage extends Storage {
// meaningful at BlockPoolSliceStorage level.
// regular start up.
if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION)
if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION)
return; // regular startup
// do upgrade
if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION) {
if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
doUpgrade(sd, nsInfo); // upgrade
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
// in readProperties(), even if rollback was not carried out or somehow
// failed.
throw new IOException("BUG: The stored LV = " + this.getLayoutVersion()
+ " is newer than the supported LV = "
+ HdfsConstants.LAYOUT_VERSION
+ " or name node LV = "
+ nsInfo.getLayoutVersion());
+ " is newer than the supported LV = "
+ HdfsConstants.DATANODE_LAYOUT_VERSION);
}
/**
@ -485,12 +521,14 @@ public class DataStorage extends Storage {
void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
// If the existing on-disk layout version supportes federation, simply
// 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
// field and overwrite the file.
LOG.info("Updating layout version from " + layoutVersion + " to "
+ nsInfo.getLayoutVersion() + " for storage " + sd.getRoot());
layoutVersion = nsInfo.getLayoutVersion();
+ HdfsConstants.DATANODE_LAYOUT_VERSION + " for storage "
+ sd.getRoot());
layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
writeProperties(sd);
return;
}
@ -498,7 +536,7 @@ public class DataStorage extends Storage {
LOG.info("Upgrading storage directory " + sd.getRoot()
+ ".\n old LV = " + this.getLayoutVersion()
+ "; old CTime = " + this.getCTime()
+ ".\n new LV = " + nsInfo.getLayoutVersion()
+ ".\n new LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+ "; new CTime = " + nsInfo.getCTime());
File curDir = sd.getCurrentDir();
@ -528,7 +566,7 @@ public class DataStorage extends Storage {
linkAllBlocks(tmpDir, bbwDir, new File(curBpDir, STORAGE_DIR_CURRENT));
// 4. Write version file under <SD>/current
layoutVersion = HdfsConstants.LAYOUT_VERSION;
layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
clusterID = nsInfo.getClusterID();
writeProperties(sd);
@ -548,7 +586,8 @@ public class DataStorage extends Storage {
* @throws IOException if the directory is not empty or it can not be removed
*/
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() ) {
if (FileUtil.list(detachDir).length != 0 ) {
@ -584,19 +623,13 @@ public class DataStorage extends Storage {
File prevDir = sd.getPreviousDir();
// This is a regular startup or a post-federation rollback
if (!prevDir.exists()) {
// The current datanode version supports federation and the layout
// version from namenode matches what the datanode supports. An invalid
// rollback may happen if namenode didn't rollback and datanode is
// 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());
if (DataNodeLayoutVersion.supports(LayoutVersion.Feature.FEDERATION,
HdfsConstants.DATANODE_LAYOUT_VERSION)) {
readProperties(sd, HdfsConstants.DATANODE_LAYOUT_VERSION);
writeProperties(sd);
LOG.info("Layout version rolled back to " +
nsInfo.getLayoutVersion() + " for storage " + sd.getRoot());
LOG.info("Layout version rolled back to "
+ HdfsConstants.DATANODE_LAYOUT_VERSION + " for storage "
+ sd.getRoot());
}
return;
}
@ -605,16 +638,17 @@ public class DataStorage extends Storage {
// We allow rollback to a state, which is either consistent with
// 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
throw new InconsistentFSStateException(sd.getRoot(),
"Cannot rollback to a newer state.\nDatanode previous state: LV = "
+ prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
+ " 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()
+ ".\n target LV = " + nsInfo.getLayoutVersion()
+ "; target CTime = " + nsInfo.getCTime());
+ ".\n target LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+ "; target CTime = " + nsInfo.getCTime());
File tmpDir = sd.getRemovedTmp();
assert !tmpDir.exists() : "removed.tmp directory must not exist.";
// rename current to tmp
@ -677,9 +711,11 @@ public class DataStorage extends Storage {
/*
* 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 {
// 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,
// then finalize it. Else finalize the corresponding BP.
for (StorageDirectory sd : storageDirs) {
@ -710,7 +746,8 @@ public class DataStorage extends Storage {
HardLink hardLink = new HardLink();
// do the link
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
linkBlocks(new File(fromDir, STORAGE_DIR_FINALIZED),
new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);

View File

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

View File

@ -20,8 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.IOException;
import java.net.SocketTimeoutException;
import java.nio.channels.AsynchronousCloseException;
import java.util.HashSet;
import java.util.Set;
import java.util.HashMap;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
@ -45,7 +44,8 @@ class DataXceiverServer implements Runnable {
private final PeerServer peerServer;
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.
@ -127,7 +127,7 @@ class DataXceiverServer implements Runnable {
@Override
public void run() {
Peer peer = null;
while (datanode.shouldRun) {
while (datanode.shouldRun && !datanode.shutdownForUpgrade) {
try {
peer = peerServer.accept();
@ -147,7 +147,7 @@ class DataXceiverServer implements Runnable {
} catch (AsynchronousCloseException ace) {
// another thread closed our listener socket - that's expected during shutdown,
// but not in other circumstances
if (datanode.shouldRun) {
if (datanode.shouldRun && !datanode.shutdownForUpgrade) {
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ace);
}
} catch (IOException ie) {
@ -170,31 +170,53 @@ class DataXceiverServer implements Runnable {
datanode.shouldRun = false;
}
}
synchronized (this) {
for (Peer p : peers) {
IOUtils.cleanup(LOG, p);
}
}
// Close the server to stop reception of more requests.
try {
peerServer.close();
closed = true;
} catch (IOException ie) {
LOG.warn(datanode.getDisplayName()
+ " :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() {
assert datanode.shouldRun == false :
"shoudRun should be set to false before killing";
assert (datanode.shouldRun == false || datanode.shutdownForUpgrade) :
"shoudRun should be set to false or restarting should be true"
+ " before killing";
try {
this.peerServer.close();
this.closed = true;
} catch (IOException ie) {
LOG.warn(datanode.getDisplayName() + ":DataXceiverServer.kill(): ", ie);
}
}
synchronized void addPeer(Peer peer) {
peers.add(peer);
synchronized void addPeer(Peer peer, Thread t) throws IOException {
if (closed) {
throw new IOException("Server closed.");
}
peers.put(peer, t);
}
synchronized void closePeer(Peer peer) {
@ -202,6 +224,31 @@ class DataXceiverServer implements Runnable {
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) {
peers.remove(peer);
}

View File

@ -412,5 +412,22 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
*/
public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
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.BufferedInputStream;
import java.io.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileWriter;
import java.io.IOException;
import java.io.InputStream;
import java.io.RandomAccessFile;
import java.util.Scanner;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DU;
@ -36,11 +39,14 @@ import org.apache.hadoop.hdfs.server.datanode.DataStorage;
import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
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.io.IOUtils;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DiskChecker;
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.
@ -56,6 +62,9 @@ class BlockPoolSlice {
private final LDir finalizedDir; // directory store Finalized replica
private final File rbwDir; // directory store RBW 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
private final DU dfsUsage;
@ -106,8 +115,21 @@ class BlockPoolSlice {
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();
// 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() {
@ -131,6 +153,79 @@ class BlockPoolSlice {
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
* the block is finalized.
@ -191,9 +286,39 @@ class BlockPoolSlice {
newReplica = new FinalizedReplica(blockId,
blockFile.length(), genStamp, volume, blockFile.getParentFile());
} else {
newReplica = new ReplicaWaitingToBeRecovered(blockId,
validateIntegrityAndSetLength(blockFile, genStamp),
genStamp, volume, blockFile.getParentFile());
boolean loadRwr = true;
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);
@ -296,6 +421,8 @@ class BlockPoolSlice {
}
void shutdown() {
saveDfsUsed();
dfsUsedSaved = true;
dfsUsage.shutdown();
}
}
}

View File

@ -153,29 +153,34 @@ class FsDatasetAsyncDiskService {
* dfsUsed statistics accordingly.
*/
void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
ExtendedBlock block) {
ExtendedBlock block, String trashDirectory) {
LOG.info("Scheduling " + block.getLocalBlock()
+ " file " + blockFile + " for deletion");
ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
volume, blockFile, metaFile, block);
volume, blockFile, metaFile, block, trashDirectory);
execute(volume.getCurrentDir(), deletionTask);
}
/** 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 {
final FsVolumeImpl volume;
final File blockFile;
final File metaFile;
final ExtendedBlock block;
final String trashDirectory;
ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile,
File metaFile, ExtendedBlock block) {
File metaFile, ExtendedBlock block, String trashDirectory) {
this.volume = volume;
this.blockFile = blockFile;
this.metaFile = metaFile;
this.block = block;
this.trashDirectory = trashDirectory;
}
@Override
@ -186,12 +191,39 @@ class FsDatasetAsyncDiskService {
+ " 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
public void run() {
long dfsBytes = blockFile.length() + metaFile.length();
if (!blockFile.delete() || (!metaFile.delete() && metaFile.exists())) {
LOG.warn("Unexpected error trying to delete block "
+ block.getBlockPoolId() + " " + block.getLocalBlock()
boolean result;
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.");
} else {
if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){

View File

@ -17,28 +17,6 @@
*/
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.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -46,37 +24,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
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.protocol.*;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
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.*;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.*;
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.DatanodeStorage;
@ -90,6 +43,15 @@ import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.ReflectionUtils;
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
* has a unique name and an extent on disk.
@ -193,6 +155,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
final DataNode datanode;
final DataStorage dataStorage;
final FsVolumeList volumes;
final FsDatasetAsyncDiskService asyncDiskService;
final FsDatasetCache cacheManager;
@ -209,6 +172,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
FsDatasetImpl(DataNode datanode, DataStorage storage, Configuration conf
) throws IOException {
this.datanode = datanode;
this.dataStorage = storage;
// The number of volumes required for operation is the total number
// of volumes minus the number of failed volumes we can tolerate.
final int volFailuresTolerated =
@ -1234,7 +1198,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
// finishes.
asyncDiskService.deleteAsync(v, f,
FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
new ExtendedBlock(bpid, invalidBlks[i]));
new ExtendedBlock(bpid, invalidBlks[i]),
dataStorage.getTrashDirectoryForBlockFile(bpid, f));
}
if (error) {
throw new IOException("Error in deleting blocks.");
@ -1762,11 +1727,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
@Override
public synchronized void addBlockPool(String bpid, Configuration conf)
public void addBlockPool(String bpid, Configuration conf)
throws IOException {
LOG.info("Adding block pool " + bpid);
volumes.addBlockPool(bpid, conf);
volumeMap.initBlockPool(bpid);
synchronized(this) {
volumes.addBlockPool(bpid, conf);
volumeMap.initBlockPool(bpid);
}
volumes.getAllVolumesMap(bpid, volumeMap);
}
@ -1895,6 +1862,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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
public RollingLogs createRollingLogs(String bpid, String prefix
) throws IOException {

View File

@ -96,10 +96,41 @@ class FsVolumeList {
}
}
void getAllVolumesMap(String bpid, ReplicaMap volumeMap) throws IOException {
void getAllVolumesMap(final String bpid, final ReplicaMap volumeMap) throws IOException {
long totalStartTime = System.currentTimeMillis();
for (FsVolumeImpl v : volumes) {
getVolumeMap(bpid, v, volumeMap);
final List<IOException> exceptions = Collections.synchronizedList(
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;
FsDatasetImpl.LOG.info("Total time to add all replicas to map: "
@ -219,4 +250,4 @@ class FsVolumeList {
}
}
}
}
}

View File

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

View File

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

View File

@ -30,12 +30,14 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.NameNodeProxies;
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.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
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.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
@ -44,8 +46,8 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
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.StandbyException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
@ -357,7 +359,7 @@ public class BackupNode extends NameNode {
} else {
nsInfo.validateStorage(storage);
}
bnImage.initEditLog();
bnImage.initEditLog(StartupOption.REGULAR);
setRegistration();
NamenodeRegistration nnReg = null;
while(!isStopRequested()) {
@ -402,9 +404,9 @@ public class BackupNode extends NameNode {
LOG.fatal(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: "
+ HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
+ HdfsConstants.NAMENODE_LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
return nsInfo;
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
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 com.google.common.collect.ComparisonChain;
@ -47,6 +48,7 @@ public class CheckpointSignature extends StorageInfo
}
CheckpointSignature(String str) {
super(NodeType.NAME_NODE);
String[] fields = str.split(FIELD_SEPARATOR);
assert fields.length == NUM_FIELDS :
"Must be " + NUM_FIELDS + " fields in CheckpointSignature";

View File

@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
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.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@ -220,9 +221,9 @@ class Checkpointer extends Daemon {
LOG.info("Unable to roll forward using only logs. Downloading " +
"image with txid " + sig.mostRecentCheckpointTxId);
MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId,
bnStorage, true);
bnImage.saveDigestAndRenameCheckpointImage(
backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId, bnStorage,
true);
bnImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE,
sig.mostRecentCheckpointTxId, downloadedHash);
lastApplied = sig.mostRecentCheckpointTxId;
needReloadImage = true;
@ -240,7 +241,8 @@ class Checkpointer extends Daemon {
if(needReloadImage) {
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());
}
rollForwardByApplyingLogs(manifest, bnImage, backupNode.getNamesystem());
@ -263,7 +265,7 @@ class Checkpointer extends Daemon {
if(cpCmd.needToReturnImage()) {
TransferFsImage.uploadImageFromStorage(
backupNode.nnHttpAddress, getImageListenAddress(),
bnStorage, txid);
bnStorage, NameNodeFile.IMAGE, txid);
}
getRemoteNamenodeProxy().endCheckpoint(backupNode.getRegistration(), sig);
@ -307,6 +309,6 @@ class Checkpointer extends Daemon {
}
LOG.info("Checkpointer about to load edits from " +
editsStreams.size() + " stream(s).");
dstImage.loadEdits(editsStreams, dstNamesystem, null);
dstImage.loadEdits(editsStreams, dstNamesystem);
}
}

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutFlags;
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.namenode.TransferFsImage.HttpGetFailedException;
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
@ -150,7 +149,8 @@ public class EditLogFileInputStream extends EditLogInputStream {
} catch (EOFException eofe) {
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 {
LayoutFlags.read(dataIn);
} catch (EOFException eofe) {
@ -329,12 +329,12 @@ public class EditLogFileInputStream extends EditLogInputStream {
throw new LogHeaderCorruptException(
"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
throw new LogHeaderCorruptException(
"Unexpected version of the file system log file: "
+ logVersion + ". Current version = "
+ HdfsConstants.LAYOUT_VERSION + ".");
+ HdfsConstants.NAMENODE_LAYOUT_VERSION + ".");
}
return logVersion;
}

View File

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

View File

@ -20,36 +20,36 @@ package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.util.ExitUtil.terminate;
import static org.apache.hadoop.util.Time.now;
import java.net.URI;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.lang.reflect.Constructor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
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.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
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.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.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.AllowSnapshotOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
@ -66,8 +66,8 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveIn
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.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.RemoveCachePoolOp;
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.RenameSnapshotOp;
@ -83,6 +83,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
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.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.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@ -91,7 +92,6 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.conf.Configuration;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -1036,6 +1036,18 @@ public class FSEditLog implements LogsPurgeable {
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.
*/
@ -1318,6 +1330,13 @@ public class FSEditLog implements LogsPurgeable {
}
}
public synchronized void discardSegments(long markerTxid)
throws IOException {
for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
jas.getManager().discardSegments(markerTxid);
}
}
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk) throws IOException {

View File

@ -37,10 +37,11 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
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.server.blockmanagement.BlockInfo;
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.namenode.FSEditLogOp.AddBlockOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
@ -68,6 +69,7 @@ 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.RenewDelegationTokenOp;
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.SetGenstampV2Op;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
@ -81,6 +83,7 @@ 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.INode.BlocksMapUpdateInfo;
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.startupprogress.Phase;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
@ -96,22 +99,30 @@ import com.google.common.base.Preconditions;
@InterfaceStability.Evolving
public class FSEditLogLoader {
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 long lastAppliedTxId;
/** Total number of end transactions loaded. */
private int totalEdits = 0;
public FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId) {
this.fsNamesys = fsNamesys;
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
* This is where we apply edits that we've been writing to disk all
* along.
*/
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId,
MetaRecoveryContext recovery) throws IOException {
StartupOption startOpt, MetaRecoveryContext recovery) throws IOException {
StartupProgress prog = NameNode.getStartupProgress();
Step step = createStartupProgressStep(edits);
prog.beginStep(Phase.LOADING_EDITS, step);
@ -119,8 +130,8 @@ public class FSEditLogLoader {
try {
long startTime = now();
FSImage.LOG.info("Start loading edits file " + edits.getName());
long numEdits = loadEditRecords(edits, false,
expectedStartingTxId, recovery);
long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
startOpt, recovery);
FSImage.LOG.info("Edits file " + edits.getName()
+ " of size " + edits.length() + " edits # " + numEdits
+ " loaded in " + (now()-startTime)/1000 + " seconds");
@ -133,8 +144,8 @@ public class FSEditLogLoader {
}
long loadEditRecords(EditLogInputStream in, boolean closeOnExit,
long expectedStartingTxId, MetaRecoveryContext recovery)
throws IOException {
long expectedStartingTxId, StartupOption startOpt,
MetaRecoveryContext recovery) throws IOException {
FSDirectory fsDir = fsNamesys.dir;
EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts =
@ -206,12 +217,23 @@ public class FSEditLogLoader {
}
}
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) {
lastInodeId = inodeId;
}
} catch (RollingUpgradeOp.RollbackException e) {
throw e;
} catch (Throwable 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 " +
"apply edit log operation " + op + ": error " +
e.getMessage(), recovery, "applying edits");
@ -237,6 +259,10 @@ public class FSEditLogLoader {
}
}
numEdits++;
totalEdits++;
} catch (RollingUpgradeOp.RollbackException e) {
LOG.info("Stopped at OP_START_ROLLING_UPGRADE for rollback.");
break;
} catch (MetaRecoveryContext.RequestStopException e) {
MetaRecoveryContext.LOG.warn("Stopped reading edit log at " +
in.getPosition() + "/" + in.length());
@ -268,7 +294,8 @@ public class FSEditLogLoader {
long inodeId = inodeIdFromOp;
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
+ " supports inodeId but gave bogus inodeId");
}
@ -285,7 +312,7 @@ public class FSEditLogLoader {
@SuppressWarnings("deprecation")
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;
if (LOG.isTraceEnabled()) {
LOG.trace("replaying edit log: " + op);
@ -693,6 +720,30 @@ public class FSEditLogLoader {
fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId);
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: {
AddCacheDirectiveInfoOp addOp = (AddCacheDirectiveInfoOp) op;
CacheDirectiveInfo result = fsNamesys.
@ -931,7 +982,7 @@ public class FSEditLogLoader {
// The editlog must be emptied by restarting the namenode, before proceeding
// with the upgrade.
if (Storage.is203LayoutVersion(logVersion)
&& logVersion != HdfsConstants.LAYOUT_VERSION) {
&& logVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
String msg = "During upgrade failed to load the editlog version "
+ logVersion + " from release 0.20.203. Please go back to the old "
+ " 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_
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_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_V2;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_NS_QUOTA;
@ -179,7 +181,12 @@ public abstract class FSEditLogOp {
inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
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) {
@ -267,8 +274,8 @@ public abstract class FSEditLogOp {
void readRpcIds(DataInputStream in, int logVersion)
throws IOException {
if (LayoutVersion.supports(Feature.EDITLOG_SUPPORT_RETRYCACHE,
logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_SUPPORT_RETRYCACHE, logVersion)) {
this.rpcClientId = FSImageSerialization.readBytes(in);
this.rpcCallId = FSImageSerialization.readInt(in);
}
@ -315,7 +322,7 @@ public abstract class FSEditLogOp {
private static List<AclEntry> read(DataInputStream in, int logVersion)
throws IOException {
if (!LayoutVersion.supports(Feature.EXTENDED_ACL, logVersion)) {
if (!NameNodeLayoutVersion.supports(Feature.EXTENDED_ACL, logVersion)) {
return null;
}
@ -480,18 +487,20 @@ public abstract class FSEditLogOp {
@Override
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();
}
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
this.inodeId = in.readLong();
} else {
// The inodeId should be updated when this editLogOp is applied
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
}
if ((-17 < logVersion && length != 4) ||
(logVersion <= -17 && length != 5 && !LayoutVersion.supports(
Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
(logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
throw new IOException("Incorrect data format." +
" logVersion is " + logVersion +
" but writables.length is " +
@ -499,7 +508,8 @@ public abstract class FSEditLogOp {
}
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.mtime = FSImageSerialization.readLong(in);
} else {
@ -507,8 +517,10 @@ public abstract class FSEditLogOp {
this.mtime = readLong(in);
}
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
this.atime = FSImageSerialization.readLong(in);
} else {
this.atime = readLong(in);
@ -517,7 +529,8 @@ public abstract class FSEditLogOp {
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);
} else {
this.blockSize = readLong(in);
@ -933,7 +946,8 @@ public abstract class FSEditLogOp {
void readFields(DataInputStream in, int logVersion)
throws IOException {
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);
} else {
this.replication = readShort(in);
@ -1024,7 +1038,8 @@ public abstract class FSEditLogOp {
@Override
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();
if (length < 3) { // trg, srcs.., timestamp
throw new IOException("Incorrect data format " +
@ -1033,7 +1048,8 @@ public abstract class FSEditLogOp {
}
this.trg = FSImageSerialization.readString(in);
int srcSize = 0;
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
srcSize = in.readInt();
} else {
srcSize = this.length - 1 - 1; // trg and timestamp
@ -1052,7 +1068,8 @@ public abstract class FSEditLogOp {
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);
} else {
this.timestamp = readLong(in);
@ -1158,7 +1175,8 @@ public abstract class FSEditLogOp {
@Override
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();
if (this.length != 3) {
throw new IOException("Incorrect data format. "
@ -1167,7 +1185,8 @@ public abstract class FSEditLogOp {
}
this.src = 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);
} else {
this.timestamp = readLong(in);
@ -1254,14 +1273,16 @@ public abstract class FSEditLogOp {
@Override
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();
if (this.length != 2) {
throw new IOException("Incorrect data format. " + "delete operation.");
}
}
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);
} else {
this.timestamp = readLong(in);
@ -1362,22 +1383,26 @@ public abstract class FSEditLogOp {
@Override
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();
}
if (-17 < logVersion && length != 2 ||
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.");
}
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
this.inodeId = FSImageSerialization.readLong(in);
} else {
// This id should be updated when this editLogOp is applied
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
}
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);
} else {
this.timestamp = readLong(in);
@ -1386,8 +1411,10 @@ public abstract class FSEditLogOp {
// The disk format stores atimes for directories as well.
// However, currently this is not being updated/used because of
// performance reasons.
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
FSImageSerialization.readLong(in);
} else {
readLong(in);
@ -1977,7 +2004,8 @@ public abstract class FSEditLogOp {
@Override
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();
if (length != 3) {
throw new IOException("Incorrect data format. " + "times operation.");
@ -1985,7 +2013,8 @@ public abstract class FSEditLogOp {
}
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.atime = FSImageSerialization.readLong(in);
} else {
@ -2094,14 +2123,16 @@ public abstract class FSEditLogOp {
@Override
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();
if (this.length != 4) {
throw new IOException("Incorrect data format. "
+ "symlink operation.");
}
}
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
this.inodeId = FSImageSerialization.readLong(in);
} else {
// This id should be updated when the editLogOp is applied
@ -2110,7 +2141,8 @@ public abstract class FSEditLogOp {
this.path = 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.atime = FSImageSerialization.readLong(in);
} else {
@ -2228,7 +2260,8 @@ public abstract class FSEditLogOp {
@Override
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();
if (this.length != 3) {
throw new IOException("Incorrect data format. " + "Rename operation.");
@ -2237,7 +2270,8 @@ public abstract class FSEditLogOp {
this.src = 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);
} else {
this.timestamp = readLong(in);
@ -2448,7 +2482,8 @@ public abstract class FSEditLogOp {
throws IOException {
this.token = new DelegationTokenIdentifier();
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);
} else {
this.expiryTime = readLong(in);
@ -2520,7 +2555,8 @@ public abstract class FSEditLogOp {
throws IOException {
this.token = new DelegationTokenIdentifier();
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);
} else {
this.expiryTime = readLong(in);
@ -3544,6 +3580,65 @@ public abstract class FSEditLogOp {
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
@ -3594,7 +3689,8 @@ public abstract class FSEditLogOp {
*/
public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
this.logVersion = logVersion;
if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) {
this.checksum = new PureJavaCrc32();
} else {
this.checksum = null;
@ -3733,7 +3829,8 @@ public abstract class FSEditLogOp {
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
op.setTransactionId(in.readLong());
} else {

View File

@ -66,8 +66,10 @@ public enum FSEditLogOpCodes {
OP_ADD_CACHE_POOL ((byte) 36),
OP_MODIFY_CACHE_POOL ((byte) 37),
OP_REMOVE_CACHE_POOL ((byte) 38),
OP_MODIFY_CACHE_DIRECTIVE ((byte) 39),
OP_MODIFY_CACHE_DIRECTIVE ((byte) 39),
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
OP_INVALID ((byte) -1);

View File

@ -21,11 +21,13 @@ import static org.apache.hadoop.util.Time.now;
import java.io.Closeable;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -41,18 +43,15 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
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.Storage.FormatConfirmable;
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.InconsistentFSStateException;
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.StorageState;
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.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@ -218,13 +217,18 @@ public class FSImage implements Closeable {
NNStorage.checkVersionUpgradable(storage.getLayoutVersion());
}
if (startOpt != StartupOption.UPGRADE
&& !RollingUpgradeStartupOption.STARTED.matches(startOpt)
&& layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
&& layoutVersion != HdfsConstants.LAYOUT_VERSION) {
&& layoutVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
throw new IOException(
"\nFile system image contains an old layout version "
+ storage.getLayoutVersion() + ".\nAn upgrade to version "
+ HdfsConstants.LAYOUT_VERSION + " is required.\n"
+ "Please restart NameNode with -upgrade option.");
+ HdfsConstants.NAMENODE_LAYOUT_VERSION + " is required.\n"
+ "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);
@ -263,7 +267,7 @@ public class FSImage implements Closeable {
// just load the image
}
return loadFSImage(target, recovery);
return loadFSImage(target, startOpt, recovery);
}
/**
@ -319,8 +323,9 @@ public class FSImage implements Closeable {
return isFormatted;
}
private void doUpgrade(FSNamesystem target) throws IOException {
// Upgrade is allowed only if there are
/** Check if upgrade is in progress. */
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
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
@ -329,15 +334,37 @@ public class FSImage implements Closeable {
"previous fs state should not exist during upgrade. "
+ "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
this.loadFSImage(target, null);
this.loadFSImage(target, null, null);
// Do upgrade for each directory
target.checkRollingUpgrade("upgrade namenode");
long oldCTime = storage.getCTime();
storage.cTime = now(); // generate new cTime for the state
int oldLV = storage.getLayoutVersion();
storage.layoutVersion = HdfsConstants.LAYOUT_VERSION;
storage.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
List<StorageDirectory> errorSDs =
Collections.synchronizedList(new ArrayList<StorageDirectory>());
@ -411,7 +438,7 @@ public class FSImage implements Closeable {
boolean canRollback = false;
FSImage prevState = new FSImage(conf);
try {
prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION;
prevState.getStorage().layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
File prevDir = sd.getPreviousDir();
@ -426,12 +453,12 @@ public class FSImage implements Closeable {
// read and verify consistency of the prev dir
prevState.getStorage().readPreviousVersionProperties(sd);
if (prevState.getLayoutVersion() != HdfsConstants.LAYOUT_VERSION) {
if (prevState.getLayoutVersion() != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
throw new IOException(
"Cannot rollback to storage version " +
prevState.getLayoutVersion() +
" 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.");
}
canRollback = true;
@ -525,7 +552,7 @@ public class FSImage implements Closeable {
// return back the real image
realImage.getStorage().setStorageInfo(ckptImage.getStorage());
realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
realImage.initEditLog();
realImage.initEditLog(StartupOption.IMPORT);
target.dir.fsImage = realImage;
realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
@ -586,13 +613,22 @@ public class FSImage implements Closeable {
* @return whether the image should be saved
* @throws IOException
*/
boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery)
private boolean loadFSImage(FSNamesystem target, StartupOption startOpt,
MetaRecoveryContext recovery)
throws IOException {
FSImageStorageInspector inspector = storage.readAndInspectDirs();
FSImageFile imageFile = null;
final boolean rollingRollback
= 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();
List<FSImageFile> imageFiles = inspector.getLatestImages();
StartupProgress prog = NameNode.getStartupProgress();
@ -604,14 +640,24 @@ public class FSImage implements Closeable {
Iterable<EditLogInputStream> editStreams = null;
initEditLog();
initEditLog(startOpt);
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT,
getLayoutVersion())) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
// 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
// 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(
imageFiles.get(0).getCheckpointTxId() + 1,
toAtLeastTxId, recovery, false);
@ -619,8 +665,7 @@ public class FSImage implements Closeable {
editStreams = FSImagePreTransactionalStorageInspector
.getEditLogStreams(storage);
}
int maxOpSize = conf.getInt(DFSConfigKeys.
DFS_NAMENODE_MAX_OP_SIZE_KEY,
int maxOpSize = conf.getInt(DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_KEY,
DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT);
for (EditLogInputStream elis : editStreams) {
elis.setMaxOpSize(maxOpSize);
@ -633,6 +678,7 @@ public class FSImage implements Closeable {
LOG.info("No edit log streams selected.");
}
FSImageFile imageFile = null;
for (int i = 0; i < imageFiles.size(); i++) {
try {
imageFile = imageFiles.get(i);
@ -650,26 +696,57 @@ public class FSImage implements Closeable {
throw new IOException("Failed to load an FSImage file!");
}
prog.endPhase(Phase.LOADING_FSIMAGE);
long txnsAdvanced = loadEdits(editStreams, target, recovery);
needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(),
txnsAdvanced);
if (!rollingRollback) {
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);
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,
FSImageFile imageFile) throws IOException {
LOG.debug("Planning to load image :\n" + imageFile);
StorageDirectory sdForProperties = imageFile.sd;
storage.readProperties(sdForProperties);
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT,
getLayoutVersion())) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
// For txid-based layout, we should have a .md5 file
// next to the image file
loadFSImage(imageFile.getFile(), target, recovery);
} else if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM,
getLayoutVersion())) {
} else if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FSIMAGE_CHECKSUM, getLayoutVersion())) {
// In 0.22, we have the checksum stored in the VERSION file.
String md5 = storage.getDeprecatedProperty(
NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY);
@ -686,11 +763,15 @@ public class FSImage implements Closeable {
}
}
public void initEditLog() {
public void initEditLog(StartupOption startOpt) {
Preconditions.checkState(getNamespaceID() != 0,
"Must know namespace ID before initting edit log");
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.recoverUnclosedStreams();
} else {
@ -722,7 +803,13 @@ public class FSImage implements Closeable {
* Load the specified list of edit files into the image.
*/
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));
StartupProgress prog = NameNode.getStartupProgress();
prog.beginPhase(Phase.LOADING_EDITS);
@ -736,7 +823,7 @@ public class FSImage implements Closeable {
LOG.info("Reading " + editIn + " expecting start txid #" +
(lastAppliedTxId + 1));
try {
loader.loadFSEdits(editIn, lastAppliedTxId + 1, recovery);
loader.loadFSEdits(editIn, lastAppliedTxId + 1, startOpt, recovery);
} finally {
// Update lastAppliedTxId even in case of error, since some ops may
// have been successfully applied before the error.
@ -750,7 +837,7 @@ public class FSImage implements Closeable {
} finally {
FSEditLog.closeAllStreams(editStreams);
// update the counts
updateCountForQuota(target.dir.rootDir);
updateCountForQuota(target.dir.rootDir);
}
prog.endPhase(Phase.LOADING_EDITS);
return lastAppliedTxId - prevLastAppliedTxId;
@ -828,9 +915,12 @@ public class FSImage implements Closeable {
*/
private void loadFSImage(File curFile, MD5Hash expectedMd5,
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);
loader.load(curFile);
target.setBlockPoolId(this.getBlockPoolID());
// Check that the image digest we loaded matches up with what
// we expected
@ -851,11 +941,11 @@ public class FSImage implements Closeable {
/**
* Save the contents of the FS image to the file.
*/
void saveFSImage(SaveNamespaceContext context, StorageDirectory sd)
throws IOException {
void saveFSImage(SaveNamespaceContext context, StorageDirectory sd,
NameNodeFile dstType) throws IOException {
long txid = context.getTxId();
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);
FSImageCompression compression = FSImageCompression.createCompression(conf);
@ -879,16 +969,19 @@ public class FSImage implements Closeable {
private class FSImageSaver implements Runnable {
private final SaveNamespaceContext context;
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.sd = sd;
this.nnf = nnf;
}
@Override
public void run() {
try {
saveFSImage(context, sd);
saveFSImage(context, sd, nnf);
} catch (SaveNamespaceCancelledException snce) {
LOG.info("Cancelled image saving for " + sd.getRoot() +
": " + snce.getMessage());
@ -924,17 +1017,18 @@ public class FSImage implements Closeable {
*/
public synchronized void saveNamespace(FSNamesystem source)
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
* 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 {
assert editLog != null : "editLog must be initialized";
LOG.info("Save namespace ...");
storage.attemptRestoreRemovedStorage();
boolean editLogWasOpen = editLog.isSegmentOpen();
@ -944,7 +1038,7 @@ public class FSImage implements Closeable {
}
long imageTxId = getLastAppliedOrWrittenTxId();
try {
saveFSImageInAllDirs(source, imageTxId, canceler);
saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
storage.writeAll();
} finally {
if (editLogWasOpen) {
@ -963,12 +1057,11 @@ public class FSImage implements Closeable {
*/
protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid)
throws IOException {
saveFSImageInAllDirs(source, txid, null);
saveFSImageInAllDirs(source, NameNodeFile.IMAGE, txid, null);
}
protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid,
Canceler canceler)
throws IOException {
private synchronized void saveFSImageInAllDirs(FSNamesystem source,
NameNodeFile nnf, long txid, Canceler canceler) throws IOException {
StartupProgress prog = NameNode.getStartupProgress();
prog.beginPhase(Phase.SAVING_CHECKPOINT);
if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
@ -986,7 +1079,7 @@ public class FSImage implements Closeable {
for (Iterator<StorageDirectory> it
= storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
StorageDirectory sd = it.next();
FSImageSaver saver = new FSImageSaver(ctx, sd);
FSImageSaver saver = new FSImageSaver(ctx, sd, nnf);
Thread saveThread = new Thread(saver, saver.toString());
saveThreads.add(saveThread);
saveThread.start();
@ -1005,11 +1098,11 @@ public class FSImage implements Closeable {
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
// old edit logs and checkpoints.
purgeOldStorage();
purgeOldStorage(nnf);
} finally {
// Notify any threads waiting on the checkpoint to be canceled
// that it is complete.
@ -1023,23 +1116,24 @@ public class FSImage implements Closeable {
* Purge any files in the storage directories that are no longer
* necessary.
*/
public void purgeOldStorage() {
void purgeOldStorage(NameNodeFile nnf) {
try {
archivalManager.purgeOldStorage();
archivalManager.purgeOldStorage(nnf);
} 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;
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
try {
renameCheckpointInDir(sd, txid);
renameImageFileInDir(sd, fromNnf, toNnf, txid, renameMD5);
} catch (IOException ioe) {
LOG.warn("Unable to rename checkpoint in " + sd, ioe);
if (al == null) {
@ -1050,7 +1144,33 @@ public class FSImage implements Closeable {
}
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,
* since the checkpoint was cancelled.
@ -1068,23 +1188,24 @@ public class FSImage implements Closeable {
storage.reportErrorsOnDirectories(al);
}
private void renameCheckpointInDir(StorageDirectory sd, long txid)
throws IOException {
File ckpt = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
File curFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid);
// renameTo fails on Windows if the destination file
// already exists.
private void renameImageFileInDir(StorageDirectory sd, NameNodeFile fromNnf,
NameNodeFile toNnf, long txid, boolean renameMD5) throws IOException {
final File fromFile = NNStorage.getStorageFile(sd, fromNnf, txid);
final File toFile = NNStorage.getStorageFile(sd, toNnf, txid);
// renameTo fails on Windows if the destination file already exists.
if(LOG.isDebugEnabled()) {
LOG.debug("renaming " + ckpt.getAbsolutePath()
+ " to " + curFile.getAbsolutePath());
LOG.debug("renaming " + fromFile.getAbsolutePath()
+ " to " + toFile.getAbsolutePath());
}
if (!ckpt.renameTo(curFile)) {
if (!curFile.delete() || !ckpt.renameTo(curFile)) {
throw new IOException("renaming " + ckpt.getAbsolutePath() + " to " +
curFile.getAbsolutePath() + " FAILED");
if (!fromFile.renameTo(toFile)) {
if (!toFile.delete() || !fromFile.renameTo(toFile)) {
throw new IOException("renaming " + fromFile.getAbsolutePath() + " to " +
toFile.getAbsolutePath() + " FAILED");
}
}
}
if (renameMD5) {
MD5FileUtils.renameMD5File(fromFile, toFile);
}
}
CheckpointSignature rollEditLog() throws IOException {
@ -1165,13 +1286,13 @@ public class FSImage implements Closeable {
* renames the image from fsimage_N.ckpt to fsimage_N and also
* saves the related .md5 file into place.
*/
public synchronized void saveDigestAndRenameCheckpointImage(
public synchronized void saveDigestAndRenameCheckpointImage(NameNodeFile nnf,
long txid, MD5Hash digest) throws IOException {
// Write and rename MD5 file
List<StorageDirectory> badSds = Lists.newArrayList();
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
File imageFile = NNStorage.getImageFile(sd, txid);
File imageFile = NNStorage.getImageFile(sd, nnf, txid);
try {
MD5FileUtils.saveMD5File(imageFile, digest);
} catch (IOException ioe) {
@ -1183,7 +1304,7 @@ public class FSImage implements Closeable {
CheckpointFaultInjector.getInstance().afterMD5Rename();
// Rename image from tmp file
renameCheckpoint(txid);
renameCheckpoint(txid, NameNodeFile.IMAGE_NEW, nnf, false);
// So long as this is the newest image available,
// advertise it as such to other checkpointers
// from now on

View File

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

View File

@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
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.CachePoolInfoProto;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
@ -281,6 +280,12 @@ public final class FSImageFormatProtobuf {
fsn.setGenerationStampV1Limit(s.getGenstampV1Limit());
fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
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 {
@ -430,7 +435,7 @@ public final class FSImageFormatProtobuf {
FileSummary.Builder b = FileSummary.newBuilder()
.setOndiskVersion(FSImageUtil.FILE_VERSION)
.setLayoutVersion(LayoutVersion.getCurrentLayoutVersion());
.setLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
codec = compression.getImageCodec();
if (codec != null) {
@ -521,6 +526,9 @@ public final class FSImageFormatProtobuf {
// from the actual saver thread, there's a potential of a
// fairness-related deadlock. See the comments on HDFS-2223.
b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID());
if (fsn.isRollingUpgrade()) {
b.setRollingUpgradeStartTime(fsn.getRollingUpgradeInfo().getStartTime());
}
NameSystemSection s = b.build();
s.writeDelimitedTo(out);

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
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.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@ -118,8 +117,9 @@ public class FSImageSerialization {
DataInput in, FSNamesystem fsNamesys, int imgVersion)
throws IOException {
byte[] name = readBytes(in);
long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
.readLong() : fsNamesys.allocateNewInodeId();
long inodeId = NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, imgVersion) ? in.readLong()
: fsNamesys.allocateNewInodeId();
short blockReplication = in.readShort();
long modificationTime = in.readLong();
long preferredBlockSize = in.readLong();

View File

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

View File

@ -25,7 +25,6 @@ import java.util.Arrays;
import org.apache.hadoop.classification.InterfaceAudience;
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.server.namenode.FSImageFormatProtobuf.Loader;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
@ -71,7 +70,7 @@ public final class FSImageUtil {
+ summary.getOndiskVersion());
}
if (!LayoutVersion.supports(Feature.PROTOBUF_FORMAT,
if (!NameNodeLayoutVersion.supports(Feature.PROTOBUF_FORMAT,
summary.getLayoutVersion())) {
throw new IOException("Unsupported layout version "
+ summary.getLayoutVersion());

View File

@ -36,10 +36,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KE
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_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_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_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
@ -175,6 +175,8 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
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.DiffReportEntry;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -196,20 +198,20 @@ import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsExce
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.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.Storage;
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.Util;
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.JournalSet.JournalAndStream;
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.ha.EditLogTailer;
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.metrics.FSNamesystemMBean;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@ -391,6 +393,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private final CacheManager cacheManager;
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
private String blockPoolId;
@ -484,7 +494,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private HAContext haContext;
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
*/
@ -605,8 +618,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @return an FSNamesystem which contains the loaded namespace
* @throws IOException if loading fails
*/
public static FSNamesystem loadFromDisk(Configuration conf)
throws IOException {
static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
checkConfiguration(conf);
FSImage fsImage = new FSImage(conf,
@ -619,10 +631,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
long loadStart = now();
String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
try {
namesystem.loadFSImage(startOpt, fsImage,
HAUtil.isHAEnabled(conf, nameserviceId));
namesystem.loadFSImage(startOpt);
} catch (IOException ioe) {
LOG.warn("Encountered exception loading fsimage", ioe);
fsImage.close();
@ -848,8 +858,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return Collections.unmodifiableList(auditLoggers);
}
void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled)
throws IOException {
private void loadFSImage(StartupOption startOpt) throws IOException {
final FSImage fsImage = getFSImage();
// format before starting up if requested
if (startOpt == StartupOption.FORMAT) {
@ -862,8 +873,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
// We shouldn't be calling saveNamespace if we've come up in standby state.
MetaRecoveryContext recovery = startOpt.createRecoveryContext();
boolean needToSave =
fsImage.recoverTransitionRead(startOpt, this, recovery) && !haEnabled;
final boolean staleImage
= 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) {
fsImage.saveNamespace(this);
} else {
@ -926,8 +944,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
nnResourceChecker = new NameNodeResourceChecker(conf);
checkAvailableResources();
assert safeMode != null &&
!safeMode.isPopulatingReplQueues();
assert safeMode != null && !isPopulatingReplQueues();
StartupProgress prog = NameNode.getStartupProgress();
prog.beginPhase(Phase.SAFEMODE);
prog.setTotal(Phase.SAFEMODE, STEP_AWAITING_REPORTED_BLOCKS,
@ -982,12 +999,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
blockManager.clearQueues();
blockManager.processAllPendingDNMessages();
if (!isInSafeMode() ||
(isInSafeMode() && safeMode.isPopulatingReplQueues())) {
// Only need to re-process the queue, If not in SafeMode.
if (!isInSafeMode()) {
LOG.info("Reprocessing replication and invalidation queues");
blockManager.processMisReplicatedBlocks();
initializeReplQueues();
}
if (LOG.isDebugEnabled()) {
LOG.debug("NameNode metadata after re-processing " +
"replication and invalidation queues during failover:\n" +
@ -1026,7 +1043,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
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
* middle of the {@link #startActiveServices()}
@ -1074,6 +1100,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
cacheManager.clearDirectiveStats();
blockManager.getDatanodeManager().clearPendingCachingCommands();
blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
// Don't want to keep replication queues when not in Active.
blockManager.clearQueues();
initializedReplQueues = false;
} finally {
writeUnlock();
}
@ -1101,6 +1130,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
}
/**
* 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
@ -4201,23 +4240,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throws IOException {
readLock();
try {
//get datanode commands
final int maxTransfer = blockManager.getMaxReplicationStreams()
- xmitsInProgress;
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
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 {
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
* resources.
@ -4501,6 +4541,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
readLock();
try {
checkOperation(OperationCategory.UNCHECKED);
if (!isInSafeMode()) {
throw new IOException("Safe mode should be turned ON "
+ "in order to create namespace image.");
@ -4602,7 +4643,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private int safeReplication;
/** threshold for populating needed replication queues */
private double replQueueThreshold;
// internal fields
/** Time when threshold was reached.
* <br> -1 safe mode is off
@ -4620,8 +4660,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private int blockReplQueueThreshold;
/** time of the last status printout */
private long lastStatusReport = 0;
/** flag indicating whether replication queues have been initialized */
boolean initializedReplQueues = false;
/** Was safemode entered automatically because available resources were low. */
private boolean resourcesLow = false;
/** Should safemode adjust its block totals as blocks come in */
@ -4681,7 +4719,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*
* @see SafeModeInfo
*/
private SafeModeInfo(boolean resourcesLow, boolean isReplQueuesInited) {
private SafeModeInfo(boolean resourcesLow) {
this.threshold = 1.5f; // this threshold can never be reached
this.datanodeThreshold = Integer.MAX_VALUE;
this.extension = Integer.MAX_VALUE;
@ -4690,7 +4728,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
this.blockTotal = -1;
this.blockSafe = -1;
this.resourcesLow = resourcesLow;
this.initializedReplQueues = isReplQueuesInited;
enter();
reportStatus("STATE* Safe mode is ON.", true);
}
@ -4704,13 +4741,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return this.reached >= 0;
}
/**
* Check if we are populating replication queues.
*/
private synchronized boolean isPopulatingReplQueues() {
return initializedReplQueues;
}
/**
* Enter safe mode.
*/
@ -4757,21 +4787,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
}
/**
* 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
* initializing replication queues.
@ -4819,7 +4834,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
if (smmthread == null && needEnter()) {
enter();
// check if we are ready to initialize replication queues
if (canInitializeReplQueues() && !isPopulatingReplQueues()) {
if (canInitializeReplQueues() && !isPopulatingReplQueues()
&& !haEnabled) {
initializeReplQueues();
}
reportStatus("STATE* Safe mode ON.", false);
@ -4844,7 +4860,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
// check if we are ready to initialize replication queues
if (canInitializeReplQueues() && !isPopulatingReplQueues()) {
if (canInitializeReplQueues() && !isPopulatingReplQueues() && !haEnabled) {
initializeReplQueues();
}
}
@ -5154,11 +5170,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
if (!shouldPopulateReplQueues()) {
return false;
}
// safeMode is volatile, and may be set to null at any time
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null)
return true;
return safeMode.isPopulatingReplQueues();
return initializedReplQueues;
}
private boolean shouldPopulateReplQueues() {
@ -5278,7 +5290,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
getEditLog().logSyncAll();
}
if (!isInSafeMode()) {
safeMode = new SafeModeInfo(resourcesLow, isPopulatingReplQueues());
safeMode = new SafeModeInfo(resourcesLow);
return;
}
if (resourcesLow) {
@ -5353,8 +5365,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
NamenodeCommand cmd = null;
try {
checkOperation(OperationCategory.CHECKPOINT);
checkNameNodeSafeMode("Checkpoint not started");
LOG.info("Start checkpoint for " + backupNode.getAddress());
cmd = getFSImage().startCheckpoint(backupNode, activeNamenode);
getEditLog().logSync();
@ -7148,6 +7160,173 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
}
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)
throws IOException {
checkOperation(OperationCategory.WRITE);

View File

@ -196,6 +196,32 @@ public class FileJournalManager implements JournalManager {
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
@ -465,6 +491,11 @@ public class FileJournalManager implements JournalManager {
renameSelf(".corrupt");
}
void moveAsideTrashFile(long markerTxid) throws IOException {
assert this.getFirstTxId() >= markerTxid;
renameSelf(".trash");
}
public void moveAsideEmptyFile() throws IOException {
assert lastTxId == HdfsConstants.INVALID_TXID;
renameSelf(".empty");
@ -489,4 +520,9 @@ public class FileJournalManager implements JournalManager {
isInProgress(), hasCorruptHeader);
}
}
@Override
public void discardSegments(long startTxid) throws IOException {
discardEditLogSegments(startTxid);
}
}

View File

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

View File

@ -65,6 +65,14 @@ public interface JournalManager extends Closeable, LogsPurgeable,
*/
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.
*/

View File

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

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.EnumSet;
import java.util.List;
import java.util.TreeSet;
@ -31,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
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.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import com.google.common.base.Preconditions;
@ -88,13 +90,35 @@ public class NNStorageRetentionManager {
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 =
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);
long minImageTxId = getImageTxIdToRetain(inspector);
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
// all txns > N. We can remove anything < N+1, since fsimage_N
// reflects the state up to and including N. However, we also

View File

@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
@ -40,6 +41,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.namenode.ha.*;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@ -64,6 +66,7 @@ import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils;
import javax.management.ObjectName;
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
@ -205,6 +208,9 @@ public class NameNode implements NameNodeStatusMXBean {
" [" + StartupOption.CLUSTERID.getName() + " cid]" +
" [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | ["
+ StartupOption.ROLLBACK.getName() + "] | ["
+ StartupOption.ROLLINGUPGRADE.getName() + " <"
+ RollingUpgradeStartupOption.DOWNGRADE.name().toLowerCase() + "|"
+ RollingUpgradeStartupOption.ROLLBACK.name().toLowerCase() + "> ] | ["
+ StartupOption.FINALIZE.getName() + "] | ["
+ StartupOption.IMPORT.getName() + "] | ["
+ StartupOption.INITIALIZESHAREDEDITS.getName() + "] | ["
@ -1119,6 +1125,10 @@ public class NameNode implements NameNodeStatusMXBean {
return null;
}
}
} else if (StartupOption.ROLLINGUPGRADE.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.ROLLINGUPGRADE;
++i;
startOpt.setRollingUpgradeStartupOption(args[i]);
} else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.ROLLBACK;
} else if (StartupOption.FINALIZE.getName().equalsIgnoreCase(cmd)) {
@ -1164,7 +1174,7 @@ public class NameNode implements NameNodeStatusMXBean {
}
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) {
@ -1194,7 +1204,7 @@ public class NameNode implements NameNodeStatusMXBean {
FSNamesystem fsn = null;
try {
fsn = FSNamesystem.loadFromDisk(conf);
fsn.saveNamespace();
fsn.getFSImage().saveNamespace(fsn);
MetaRecoveryContext.LOG.info("RECOVERY COMPLETE");
} catch (IOException e) {
MetaRecoveryContext.LOG.info("RECOVERY FAILED: caught exception", e);
@ -1210,6 +1220,7 @@ public class NameNode implements NameNodeStatusMXBean {
public static NameNode createNameNode(String argv[], Configuration conf)
throws IOException {
LOG.info("createNameNode " + Arrays.asList(argv));
if (conf == null)
conf = new HdfsConfiguration();
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 java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
/**
* This is the JMX management interface for namenode information
@ -78,7 +79,14 @@ public interface NameNodeMXBean {
* @return true, if upgrade is finalized
*/
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
* temporary files on the local file system

View File

@ -37,6 +37,7 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.ContentSummary;
@ -48,7 +49,6 @@ import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
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.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission;
@ -64,12 +64,11 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.protocol.AclException;
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.CacheDirectiveEntry;
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.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -78,6 +77,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
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.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
@ -104,6 +105,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
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.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@ -871,6 +873,21 @@ class NameNodeRpcServer implements NamenodeProtocols {
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
public void metaSave(String filename) throws IOException {
namesystem.metaSave(filename);
@ -969,7 +986,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
@Override // DatanodeProtocol
public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
throws IOException {
verifyLayoutVersion(nodeReg.getVersion());
verifySoftwareVersion(nodeReg);
namesystem.registerDatanode(nodeReg);
return nodeReg;
@ -1071,13 +1087,29 @@ class NameNodeRpcServer implements NamenodeProtocols {
* @param nodeReg node registration
* @throws UnregisteredNodeException if the registration is invalid
*/
void verifyRequest(NodeRegistration nodeReg) throws IOException {
verifyLayoutVersion(nodeReg.getVersion());
if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) {
LOG.warn("Invalid registrationID - expected: "
+ namesystem.getRegistrationID() + " received: "
+ nodeReg.getRegistrationID());
throw new UnregisteredNodeException(nodeReg);
private void verifyRequest(NodeRegistration nodeReg) throws IOException {
// verify registration ID
final String id = nodeReg.getRegistrationID();
final String expectedID = namesystem.getRegistrationID();
if (!expectedID.equals(id)) {
LOG.warn("Registration IDs mismatched: the "
+ 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 @@ class NameNodeRpcServer implements NamenodeProtocols {
* @throws IOException
*/
void verifyLayoutVersion(int version) throws IOException {
if (version != HdfsConstants.LAYOUT_VERSION)
throw new IncorrectVersionException(version, "data node");
if (version != HdfsConstants.NAMENODE_LAYOUT_VERSION)
throw new IncorrectVersionException(
HdfsConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
}
private void verifySoftwareVersion(DatanodeRegistration dnReg)

View File

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

View File

@ -23,7 +23,6 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.net.HttpURLConnection;
import java.net.InetSocketAddress;
import java.net.URL;
import java.security.DigestInputStream;
import java.security.MessageDigest;
@ -39,10 +38,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
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.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@ -50,11 +45,13 @@ import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
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.NameNodeFile;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
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 com.google.common.annotations.VisibleForTesting;
@ -90,10 +87,9 @@ public class TransferFsImage {
null, false);
}
public static MD5Hash downloadImageToStorage(
URL fsName, long imageTxId, Storage dstStorage, boolean needDigest)
throws IOException {
String fileid = GetImageServlet.getParamStringForImage(
public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId,
Storage dstStorage, boolean needDigest) throws IOException {
String fileid = GetImageServlet.getParamStringForImage(null,
imageTxId, dstStorage);
String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
@ -166,14 +162,14 @@ public class TransferFsImage {
* @param myNNAddress the host/port where the local node is running an
* HTTPServer hosting GetImageServlet
* @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
*/
public static void uploadImageFromStorage(URL fsName,
URL myNNAddress,
Storage storage, long txid) throws IOException {
public static void uploadImageFromStorage(URL fsName, URL myNNAddress,
Storage storage, NameNodeFile nnf, long txid) throws IOException {
String fileid = GetImageServlet.getParamStringToPutImage(
txid, myNNAddress, storage);
String fileid = GetImageServlet.getParamStringToPutImage(nnf, txid,
myNNAddress, storage);
// this doesn't directly upload an image, but rather asks the NN
// to connect back to the 2NN to download the specified image.
try {

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.NameNodeProxies;
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.namenode.EditLogInputStream;
import org.apache.hadoop.hdfs.server.namenode.FSImage;
@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
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.NamespaceInfo;
import org.apache.hadoop.hdfs.tools.DFSHAAdmin;
@ -152,9 +154,9 @@ public class BootstrapStandby implements Tool, Configurable {
}
if (!checkLayoutVersion(nsInfo)) {
LOG.fatal("Layout version on remote node (" +
nsInfo.getLayoutVersion() + ") does not match " +
"this node's layout version (" + HdfsConstants.LAYOUT_VERSION + ")");
LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
+ ") does not match " + "this node's layout version ("
+ HdfsConstants.NAMENODE_LAYOUT_VERSION + ")");
return ERR_CODE_INVALID_VERSION;
}
@ -192,7 +194,7 @@ public class BootstrapStandby implements Tool, Configurable {
FSImage image = new FSImage(conf);
try {
image.getStorage().setStorageInfo(storage);
image.initEditLog();
image.initEditLog(StartupOption.REGULAR);
assert image.getEditLog().isOpenForRead() :
"Expected edit log to be open for read";
@ -206,9 +208,9 @@ public class BootstrapStandby implements Tool, Configurable {
// Download that checkpoint into our storage directories.
MD5Hash hash = TransferFsImage.downloadImageToStorage(
otherHttpAddr, imageTxId,
storage, true);
image.saveDigestAndRenameCheckpointImage(imageTxId, hash);
otherHttpAddr, imageTxId, storage, true);
image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId,
hash);
} catch (IOException ioe) {
image.close();
throw ioe;
@ -256,7 +258,7 @@ public class BootstrapStandby implements Tool, Configurable {
}
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 {

View File

@ -224,7 +224,7 @@ public class EditLogTailer {
// disk are ignored.
long editsLoaded = 0;
try {
editsLoaded = image.loadEdits(streams, namesystem, null);
editsLoaded = image.loadEdits(streams, namesystem);
} catch (EditLogInputException elie) {
editsLoaded = elie.getNumEditsLoaded();
throw elie;

View File

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

View File

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

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.namenode.NNStorage;
import org.apache.hadoop.util.VersionInfo;
@ -41,13 +42,14 @@ public class NamespaceInfo extends StorageInfo {
String softwareVersion;
public NamespaceInfo() {
super();
super(NodeType.NAME_NODE);
buildVersion = null;
}
public NamespaceInfo(int nsID, String clusterID, String bpID,
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;
this.buildVersion = buildVersion;
this.softwareVersion = softwareVersion;

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.tools;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.URL;
import java.security.PrivilegedExceptionAction;
@ -47,9 +48,12 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
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.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
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.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
@ -64,6 +68,8 @@ import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner;
import com.google.common.base.Preconditions;
/**
* This class provides some DFS administrative access shell commands.
*/
@ -271,7 +277,71 @@ public class DFSAdmin extends FsShell {
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.
*/
@ -576,9 +646,11 @@ public class DFSAdmin extends FsShell {
"\t[" + ClearQuotaCommand.USAGE +"]\n" +
"\t[" + SetSpaceQuotaCommand.USAGE + "]\n" +
"\t[" + ClearSpaceQuotaCommand.USAGE +"]\n" +
"\t[-finalizeUpgrade]\n" +
"\t[" + RollingUpgradeCommand.USAGE +"]\n" +
"\t[-refreshServiceAcl]\n" +
"\t[-refreshUserToGroupsMappings]\n" +
"\t[refreshSuperUserGroupsConfiguration]\n" +
"\t[-refreshSuperUserGroupsConfiguration]\n" +
"\t[-refreshCallQueue]\n" +
"\t[-printTopology]\n" +
"\t[-refreshNamenodes datanodehost:port]\n"+
@ -587,6 +659,8 @@ public class DFSAdmin extends FsShell {
"\t[-fetchImage <local directory>]\n" +
"\t[-allowSnapshot <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";
String report ="-report: \tReports basic filesystem information and statistics.\n";
@ -685,6 +759,18 @@ public class DFSAdmin extends FsShell {
String disallowSnapshot = "-disallowSnapshot <snapshotDir>:\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" +
"\t\tis specified.\n";
@ -703,6 +789,8 @@ public class DFSAdmin extends FsShell {
System.out.println(refreshNodes);
} else if ("finalizeUpgrade".equals(cmd)) {
System.out.println(finalizeUpgrade);
} else if (RollingUpgradeCommand.matches("-"+cmd)) {
System.out.println(RollingUpgradeCommand.DESCRIPTION);
} else if ("metasave".equals(cmd)) {
System.out.println(metaSave);
} else if (SetQuotaCommand.matches("-"+cmd)) {
@ -735,6 +823,10 @@ public class DFSAdmin extends FsShell {
System.out.println(allowSnapshot);
} else if ("disallowSnapshot".equalsIgnoreCase(cmd)) {
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)) {
System.out.println(help);
} else {
@ -746,6 +838,7 @@ public class DFSAdmin extends FsShell {
System.out.println(restoreFailedStorage);
System.out.println(refreshNodes);
System.out.println(finalizeUpgrade);
System.out.println(RollingUpgradeCommand.DESCRIPTION);
System.out.println(metaSave);
System.out.println(SetQuotaCommand.DESCRIPTION);
System.out.println(ClearQuotaCommand.DESCRIPTION);
@ -762,6 +855,8 @@ public class DFSAdmin extends FsShell {
System.out.println(fetchImage);
System.out.println(allowSnapshot);
System.out.println(disallowSnapshot);
System.out.println(shutdownDatanode);
System.out.println(getDatanodeInfo);
System.out.println(help);
System.out.println();
ToolRunner.printGenericCommandUsage(System.out);
@ -980,6 +1075,9 @@ public class DFSAdmin extends FsShell {
} else if ("-finalizeUpgrade".equals(cmd)) {
System.err.println("Usage: java DFSAdmin"
+ " [-finalizeUpgrade]");
} else if (RollingUpgradeCommand.matches(cmd)) {
System.err.println("Usage: java DFSAdmin"
+ " [" + RollingUpgradeCommand.USAGE+"]");
} else if ("-metasave".equals(cmd)) {
System.err.println("Usage: java DFSAdmin"
+ " [-metasave filename]");
@ -1034,6 +1132,7 @@ public class DFSAdmin extends FsShell {
System.err.println(" [-restoreFailedStorage true|false|check]");
System.err.println(" [-refreshNodes]");
System.err.println(" [-finalizeUpgrade]");
System.err.println(" ["+RollingUpgradeCommand.USAGE+"]");
System.err.println(" [-metasave filename]");
System.err.println(" [-refreshServiceAcl]");
System.err.println(" [-refreshUserToGroupsMappings]");
@ -1048,6 +1147,8 @@ public class DFSAdmin extends FsShell {
System.err.println(" ["+ClearSpaceQuotaCommand.USAGE+"]");
System.err.println(" [-setBalancerBandwidth <bandwidth in bytes per second>]");
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();
ToolRunner.printGenericCommandUsage(System.err);
@ -1119,6 +1220,11 @@ public class DFSAdmin extends FsShell {
printUsage(cmd);
return exitCode;
}
} else if (RollingUpgradeCommand.matches(cmd)) {
if (argv.length < 1 || argv.length > 2) {
printUsage(cmd);
return exitCode;
}
} else if ("-metasave".equals(cmd)) {
if (argv.length != 2) {
printUsage(cmd);
@ -1159,6 +1265,16 @@ public class DFSAdmin extends FsShell {
printUsage(cmd);
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
@ -1194,6 +1310,8 @@ public class DFSAdmin extends FsShell {
exitCode = refreshNodes();
} else if ("-finalizeUpgrade".equals(cmd)) {
exitCode = finalizeUpgrade();
} else if (RollingUpgradeCommand.matches(cmd)) {
exitCode = RollingUpgradeCommand.run(getDFS(), argv, i);
} else if ("-metasave".equals(cmd)) {
exitCode = metaSave(argv, i);
} else if (ClearQuotaCommand.matches(cmd)) {
@ -1222,6 +1340,10 @@ public class DFSAdmin extends FsShell {
exitCode = setBalancerBandwidth(argv, i);
} else if ("-fetchImage".equals(cmd)) {
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)) {
if (i < argv.length) {
printHelp(argv[i]);
@ -1306,6 +1428,35 @@ public class DFSAdmin extends FsShell {
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.
* @param argv Command line parameters.

View File

@ -30,10 +30,10 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
import org.apache.hadoop.hdfs.protocol.LayoutFlags;
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.server.namenode.FSImageSerialization;
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.io.Text;
import org.apache.hadoop.io.WritableUtils;
@ -158,7 +158,8 @@ class ImageLoaderCurrent implements ImageLoader {
imageVersion = in.readInt();
if( !canLoadVersion(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);
}
@ -169,22 +170,25 @@ class ImageLoaderCurrent implements ImageLoader {
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_V1_LIMIT, 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());
}
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());
}
boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
imageVersion);
boolean supportSnapshot = NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SNAPSHOT, imageVersion);
if (supportSnapshot) {
v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt());
int numSnapshots = in.readInt();
@ -194,7 +198,8 @@ class ImageLoaderCurrent implements ImageLoader {
}
}
if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FSIMAGE_COMPRESSION, imageVersion)) {
boolean isCompressed = in.readBoolean();
v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
if (isCompressed) {
@ -216,11 +221,13 @@ class ImageLoaderCurrent implements ImageLoader {
processINodesUC(in, v, skipBlocks);
if (LayoutVersion.supports(Feature.DELEGATION_TOKEN, imageVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.DELEGATION_TOKEN, imageVersion)) {
processDelegationTokens(in, v);
}
if (LayoutVersion.supports(Feature.CACHING, imageVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.CACHING, imageVersion)) {
processCacheManagerState(in, v);
}
v.leaveEnclosingElement(); // FSImage
@ -323,7 +330,8 @@ class ImageLoaderCurrent implements ImageLoader {
String n = new String(name, "UTF8");
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();
v.visit(ImageElement.INODE_ID, inodeId);
}
@ -443,7 +451,8 @@ class ImageLoaderCurrent implements ImageLoader {
v.visitEnclosingElement(ImageElement.INODES,
ImageElement.NUM_INODES, numInodes);
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
if (NameNodeLayoutVersion.supports(
LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
if (!supportSnapshot) {
processLocalNameINodes(in, v, numInodes, skipBlocks);
} else {
@ -584,7 +593,8 @@ class ImageLoaderCurrent implements ImageLoader {
if (!useRoot) {
if (in.readBoolean()) {
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);
} else {
processINode(in, v, true, currentINodeName, true);
@ -678,10 +688,10 @@ class ImageLoaderCurrent implements ImageLoader {
private void processINode(DataInputStream in, ImageVisitor v,
boolean skipBlocks, String parentName, boolean isSnapshotCopy)
throws IOException {
boolean supportSnapshot =
LayoutVersion.supports(Feature.SNAPSHOT, imageVersion);
boolean supportInodeId =
LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion);
boolean supportSnapshot = NameNodeLayoutVersion.supports(
LayoutVersion.Feature.SNAPSHOT, imageVersion);
boolean supportInodeId = NameNodeLayoutVersion.supports(
LayoutVersion.Feature.ADD_INODE_ID, imageVersion);
v.visitEnclosingElement(ImageElement.INODE);
final String pathName = readINodePath(in, parentName);
@ -694,7 +704,8 @@ class ImageLoaderCurrent implements ImageLoader {
}
v.visit(ImageElement.REPLICATION, in.readShort());
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.BLOCK_SIZE, in.readLong());
int numBlocks = in.readInt();
@ -723,7 +734,8 @@ class ImageLoaderCurrent implements ImageLoader {
dirNodeMap.put(inodeId, pathName);
}
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);
if (supportSnapshot) {
boolean snapshottable = in.readBoolean();
@ -771,7 +783,8 @@ class ImageLoaderCurrent implements ImageLoader {
v.visit(ImageElement.INODE_PATH, pathName);
processPermission(in, v);
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()));
}
@ -800,7 +813,8 @@ class ImageLoaderCurrent implements ImageLoader {
v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong());
if (in.readBoolean()) {
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);
} else {
processINode(in, v, true, currentINodeName, true);

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.util;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
@ -65,23 +66,17 @@ public abstract class MD5FileUtils {
}
/**
* Read the md5 checksum stored alongside the given file, or null
* if no md5 is stored.
* Read the md5 file stored alongside the given data file
* and match the md5 file content.
* @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 {
File md5File = getDigestFileForFile(dataFile);
String md5Line;
if (!md5File.exists()) {
return null;
}
private static Matcher readStoredMd5(File md5File) throws IOException {
BufferedReader reader =
new BufferedReader(new InputStreamReader(new FileInputStream(
md5File), Charsets.UTF_8));
String md5Line;
try {
md5Line = reader.readLine();
if (md5Line == null) { md5Line = ""; }
@ -94,9 +89,24 @@ public abstract class MD5FileUtils {
Matcher matcher = LINE_REGEX.matcher(md5Line);
if (!matcher.matches()) {
throw new IOException("Invalid MD5 file at " + md5File
+ " (does not match expected pattern)");
throw new IOException("Invalid MD5 file " + md5File + ": the content \""
+ 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);
File referencedFile = new File(matcher.group(2));
@ -135,15 +145,37 @@ public abstract class MD5FileUtils {
*/
public static void saveMD5File(File dataFile, MD5Hash digest)
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);
String digestString = StringUtils.byteToHexString(
digest.getDigest());
String md5Line = digestString + " *" + dataFile.getName() + "\n";
AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File);
afos.write(md5Line.getBytes(Charsets.UTF_8));
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 ];
}
/**
* 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.
* See the request and response for details of rpc call.
@ -164,4 +186,10 @@ service ClientDatanodeProtocolService {
*/
rpc getHdfsBlockLocations(GetHdfsBlockLocationsRequestProto)
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
}
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 {
required string path = 1;
optional string cookie = 2;
@ -659,6 +680,8 @@ service ClientNamenodeProtocol {
rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
returns(FinalizeUpgradeResponseProto);
rpc rollingUpgrade(RollingUpgradeRequestProto)
returns(RollingUpgradeResponseProto);
rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
returns(ListCorruptFileBlocksResponseProto);
rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);

View File

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

View File

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

View File

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

View File

@ -71,6 +71,7 @@ message NameSystemSection {
optional uint64 genstampV1Limit = 4;
optional uint64 lastAllocatedBlockId = 5;
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
}
/**
* Datanode local information
*/
message DatanodeLocalInfoProto {
required string softwareVersion = 1;
required string configVersion = 2;
required uint64 uptime = 3;
}
/**
* DatanodeInfo array
*/
@ -459,4 +468,9 @@ message SnapshotInfoProto {
// 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