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:
parent
0663d54767
commit
134a29f226
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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.");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()) + ':' +
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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){
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -97,4 +97,9 @@ class BackupJournalManager implements JournalManager {
|
|||
public String toString() {
|
||||
return "BackupJournalManager";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void discardSegments(long startTxId) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 "
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 " +
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -71,6 +71,7 @@ message NameSystemSection {
|
|||
optional uint64 genstampV1Limit = 4;
|
||||
optional uint64 lastAllocatedBlockId = 5;
|
||||
optional uint64 transactionId = 6;
|
||||
optional uint64 rollingUpgradeStartTime = 7;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue