Java tutorial
/** * 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.io.IOException; import java.io.BufferedOutputStream; import java.io.BufferedInputStream; import java.io.DataOutputStream; import java.io.DataInputStream; import java.io.File; import java.io.FileOutputStream; import java.io.FileInputStream; import java.io.PrintWriter; import java.io.StringWriter; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.ServerSocket; import java.net.SocketTimeoutException; import java.net.URI; import java.net.UnknownHostException; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.ArrayList; import java.util.Collection; import java.util.Map; import javax.management.NotCompliantMBeanException; import javax.management.StandardMBean; import org.apache.hadoop.ipc.*; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.ReconfigurationException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; import org.apache.hadoop.metrics.util.MBeanUtil; import org.apache.hadoop.hdfs.AvatarFailoverSnapshot; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.FastProtocolHDFS; import org.apache.hadoop.hdfs.FastWritableHDFS; import org.apache.hadoop.hdfs.FileStatusExtended; import org.apache.hadoop.hdfs.OpenFilesInfo; import org.apache.hadoop.hdfs.protocol.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.FSConstants; import org.apache.hadoop.util.FlushableLogger; import org.apache.hadoop.util.InjectionHandler; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.hdfs.protocol.AvatarProtocol; import org.apache.hadoop.hdfs.protocol.AvatarConstants.Avatar; import org.apache.hadoop.hdfs.protocol.AvatarConstants.StartupOption; import org.apache.hadoop.hdfs.protocol.AvatarConstants.InstanceId; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.protocol.AvatarDatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.BlockFlags; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockReport; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.IncrementalBlockReport; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.ReceivedBlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsConstants; import org.apache.hadoop.hdfs.server.common.HdfsConstants.Transition; import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.datanode.DatanodeProtocols; import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo; import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.NameNodeKey; import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType; import org.apache.hadoop.hdfs.server.namenode.metrics.AvatarNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.metrics.AvatarNodeStatusMBean; import org.apache.hadoop.hdfs.util.InjectionEvent; import org.apache.hadoop.hdfs.util.LightWeightBitSet; /** * This is an implementation of the AvatarNode, a hot * standby for the NameNode. * This is really cool, believe me! * The AvatarNode has two avatars.. the Standby avatar and the Active * avatar. * * In the Standby avatar, the AvatarNode is consuming transaction logs * generated by the primary (via a transaction log stored in a shared device). * Typically, the primary Namenode is writing transactions to a NFS filesystem * and the Standby is reading the log from the same NFS filesystem. The * Standby is also making periodic checkpoints to the primary namenode. * * A manual command can switch the AvatarNode from the Standby avatar * to the Active avatar. In the Active avatar, the AvatarNode performs precisely * the same functionality as a real usual Namenode. The switching from * Standby avatar to the Active avatar is fast and can typically occur * within seconds. * * Typically, an adminstrator will run require two shared mount points for * transaction logs. It has to be set in fs.name.dir.shared0 and * fs.name.dir.shared1 (similarly for edits). Then the adminstrator starts * the AvatarNode on two different machines as follows: * * bin/hadoop org.apache.hadoop.hdfs.server.namenode.AvatarNode -zero -active * bin/hadoop org.apache.hadoop.hdfs.server.namenode.AvatarNode -one -standby * The first AvatarNode uses fs.name.dir.shared0 while the second * AvatarNode uses fs.name.dir.shared1 to write its transaction logs. * Also, at startup, the first instance is the primary Namenode and the * second instance is the Standby * * After a while, the adminstrator decides to change the avatar of the * second instance to Active. In this case, he/she has to first ensure that the * first instance is really really dead. This code does not handle the * split-brain scenario where there are two active namenodes in one cluster. * */ public class AvatarNode extends NameNode implements AvatarProtocol, AvatarNodeStatusMBean { static { Configuration.addDefaultResource("avatar-default.xml"); Configuration.addDefaultResource("avatar-site.xml"); } public static final Log LOG = LogFactory.getLog(AvatarNode.class.getName()); // immediate flush logger private static final Log FLOG = FlushableLogger.getLogger(LOG); private static final int INVALIDATES_CLEANUP_INTERVAL = 60 * 1000; public static final String FAILOVER_SNAPSHOT_FILE = "failover_snapshot_file"; // The instanceId is assigned at startuptime and does not change for // the lifetime of the Node. The adminstrator has to name each instance // of the AvatarNode with a different instanceId. The node number is used // by the AvaterNode to determine which shared devices it should use to // checkpoint the image. // private InstanceId instance; // The txid the fsimage was sync-ed from the remote AvatarNode volatile private long startCheckpointTxId; private Server server; /** RPC server */ private InetSocketAddress serverAddress; /** RPC server address */ private volatile Avatar currentAvatar; // the current incarnation of this node private Standby standby; // the standby object private Configuration confg; // config for the standby namenode private Configuration startupConf; // config for the namenode private Thread standbyThread; // the standby daemon thread private Cleaner cleaner; // The thread cleaning up invalidates and mis-replicated blocks private Thread cleanerThread; private RunInfo runInfo; private long sessionId; private StandbySafeMode standbySafeMode; private volatile boolean isInitialized = false; protected final boolean enableTestFramework; protected final boolean enableTestFrameworkFsck; private String failoverFsck = ""; private String oldPrimaryFsck = ""; private volatile FailoverState failoverState = FailoverState.BEFORE_FAILOVER; private final AvatarNodeMetrics metrics; static public enum FailoverState { BEFORE_FAILOVER("BeforeFailover"), START_FAILOVER("StartFailover"), FAILED_FAILOVER( "FailedFailover"), AWAIT_FAILOVER("AwaitFailover"), PERFORM_FAILOVER("PerformFailover"); private String name = null; private FailoverState(String arg) { this.name = arg; } @Override public String toString() { return name; } } /** * The startup Conf is the original configuration of the AvatarNode. It is used by the * secondary namenode to talk to the primary namenode. * The conf is the modified configuration that is used by the standby namenode */ AvatarNode(Configuration startupConf, Configuration conf, StartupInfo startInfo, RunInfo runInfo, long sessionId, InetSocketAddress nameNodeAddr, NamenodeProtocol primaryNamenode) throws IOException { super(conf); // wrap namenode metrics this.metrics = new AvatarNodeMetrics(super.getNameNodeMetrics()); // check if we talk to primary if (startInfo.isStandby && (nameNodeAddr == null || primaryNamenode == null)) { throw new IOException("RPC to primary namenode not initialized"); } this.sessionId = sessionId; this.runInfo = runInfo; this.instance = startInfo.instance; this.enableTestFramework = (conf.getFloat("dfs.avatarnode.failover.sample.percent", 0.0f) != 0.0f); this.enableTestFrameworkFsck = (conf.getBoolean("dfs.avatarnode.failover.fsck", false)); // if we are starting as the standby then // record the fstime of the checkpoint that we are about to sync from if (startInfo.isStandby) { // Set the checkpoint time to the fstime of the image and edits // that were copied setStartCheckpointTxId(namesystem.getFSImage().storage.getMostRecentCheckpointTxId()); } initialize(conf); currentAvatar = startInfo.isStandby ? Avatar.STANDBY : Avatar.ACTIVE; this.startupConf = startupConf; this.confg = conf; this.nameserviceId = startInfo.serviceName; if (currentAvatar == Avatar.STANDBY) { // Verify we have the correct safemode. SafeModeInfo safeMode = super.namesystem.getSafeModeInstance(); if (safeMode == null || !(safeMode instanceof StandbySafeMode)) { throw new IOException("Invalid safe mode for Standby Avatar : " + safeMode + " Standby Avatar should be using " + StandbySafeMode.class + " as its dfs.safemode.impl"); } standbySafeMode = (StandbySafeMode) safeMode; // Standby has a different property for the max buffered transactions // to replay the log faster int maxStandbyBufferedTransactions = confg.getInt("dfs.max.standby.buffered.transactions", HdfsConstants.DEFAULT_MAX_BUFFERED_TRANSACTIONS); FSEditLog.setMaxBufferedTransactions(maxStandbyBufferedTransactions); // Create a standby object which does the actual work of // processing transactions from the primary and checkpointing standby = new Standby(this, startupConf, confg, nameNodeAddr, primaryNamenode); standbyThread = new Thread(standby); standbyThread.setName("Standby"); standbyThread.start(); cleaner = new Cleaner(); cleanerThread = new Thread(cleaner); cleanerThread.start(); } isInitialized = true; } protected void setFailoverFsck(String fsck) { failoverFsck = fsck; } /** * Wait for the StandbyNode to exit. If it does, then stop the underlying namenode. */ public void waitForRestart() { if (standbyThread != null) { try { // if this is the standby avatarnode, then wait for the Standby to exit standbyThread.join(); } catch (InterruptedException ie) { //eat it up } standbyThread = null; LOG.info("waitForRestart: Standby thread exited."); InjectionHandler.processEvent(InjectionEvent.AVATARNODE_WAIT_FOR_RESTART); while (failoverState == FailoverState.START_FAILOVER || failoverState == FailoverState.AWAIT_FAILOVER) { LOG.info("Current state : " + failoverState + ". Waiting for failover ...."); try { Thread.sleep(1000); } catch (InterruptedException ie) { throw new RuntimeException("waitForRestart() interrupted"); } } // if we are still in standbymode, that means we need to restart from // scratch. if (getAvatar() == Avatar.STANDBY) { runInfo.isRunning = false; LOG.info("waitForRestart Stopping encapsulated namenode."); super.stop(); // terminate encapsulated namenode super.join(); // wait for encapsulated namenode to exit shutdownStandby(); LOG.info("waitForRestart exiting"); return; } } super.join(); // wait for encapsulated namenode } public void registerMBean() { StandardMBean avatarNodeBean; try { avatarNodeBean = new StandardMBean(this, AvatarNodeStatusMBean.class); MBeanUtil.registerMBean("AvatarNode", "AvatarNodeState", avatarNodeBean); } catch (NotCompliantMBeanException mex) { LOG.error("Error registering mbean with JMX", mex); } } public AvatarNodeMetrics getAvatarNodeMetrics() { return metrics; } @Override public String getInstance() { return this.instance.toString(); } @Override public String getState() { return this.currentAvatar.toString(); } @Override public long getLagBytes() { return standby == null ? 0 : standby.getLagBytes(); } public Configuration getStartupConf() { return this.startupConf; } /** * Initialize AvatarNode * @param conf the configuration */ private void initialize(Configuration conf) throws IOException { InetSocketAddress socAddr = AvatarNode.getAddress(conf); int handlerCount = conf.getInt("hdfs.avatarnode.handler.count", 3); // create rpc server // no point to deserialize job file in Avatar node. this.server = RPC.getServer(this, socAddr.getAddress().getHostAddress(), socAddr.getPort(), handlerCount, false, conf, false); // The rpc-server port can be ephemeral... ensure we have the // correct info this.serverAddress = this.server.getListenerAddress(); LOG.info("AvatarNode up at: " + this.serverAddress); this.registerMBean(); this.server.start(); } /** * If the specified protocol is AvatarProtocol, then return the * AvatarProtocol version id, otherwise delegate to the underlying * namenode. */ public long getProtocolVersion(String protocol, long clientVersion) throws IOException { if (protocol.equals(AvatarProtocol.class.getName())) { return AvatarProtocol.versionID; } else { return super.getProtocolVersion(protocol, clientVersion); } } // // methods to support Avatar Protocol // /** * @inheritDoc */ public synchronized Avatar getAvatar() { return currentAvatar; } /** * @inheritDoc */ public Avatar reportAvatar() { return currentAvatar; } /** * @inheritDoc */ public boolean isInitialized() throws IOException { // check permissions, if this fails, the failover will not // progress, and this will prevent shutting down the primary avatar super.namesystem.checkSuperuserPrivilege(); if (currentAvatar == Avatar.ACTIVE) { // check for upgrades if (!getFSImage().isUpgradeFinalized()) { throw new IOException("Failover: Upgrade must be finalized before failover"); } } else /* STANDBY */ { // check if there are problems with checkpointing if (standby.getNumCheckpointFailures() > 0) { throw new IOException("Failover: Standby has problems with checkpointing"); } } // for both check if the node is initialized if (!isInitialized) { throw new IOException("Failover: Standby is not initialized"); } // for both check active edit streams verifyEditStreams(); return true; } /** * @inheritDoc */ protected boolean shouldCheckHeartbeat() { if (currentAvatar == Avatar.ACTIVE) { return super.shouldCheckHeartbeat(); } return true; } private static class ShutdownAvatarThread extends Thread { private final AvatarNode node; public ShutdownAvatarThread(AvatarNode node) { this.node = node; } public void run() { try { node.runInfo.shutdown = true; LOG.info("Failover: Shutdown thread for " + node.currentAvatar + " starting..."); if (node.currentAvatar == Avatar.STANDBY) { // make sure that all transactions are consumed try { // do not recover the unclosed segment here. node.standby.quiesce(FSEditLogLoader.TXID_IGNORE, false); } catch (Throwable e) { LOG.warn("Failover: standby error ", e); } } // Need to stop RPC threads before capturing any final data about the // primary avatar. node.stopRPC(false); String fsck = ""; try { if (node.enableTestFramework && node.enableTestFrameworkFsck) { LOG.info("Failover: Test framework - running fsck"); fsck = node.runFailoverFsck(); LOG.info("Failover: Test framework - fsck done"); } } catch (IOException e) { /*ignore*/ } // check if the shared journal is still available node.verifyEditStreams(); // stop the node (namesystem, fsimage, editlog, etc.) node.stop(); node.join(); // wait for encapsulated namenode to exit long totalBlocks = node.namesystem.getBlocksTotal(); if (InjectionHandler.falseCondition(InjectionEvent.AVATARNODE_SHUTDOWN, totalBlocks)) { // simulate crash return; } if (node.currentAvatar == Avatar.STANDBY) { node.shutdownStandby(); } else if (node.currentAvatar == Avatar.ACTIVE) { // If we are the primary we need to sync our last transaction id to // zookeeper. node.writeFailoverTestData(fsck); AvatarNodeZkUtil.writeLastTxidToZookeeper(node.getLastWrittenTxId(), totalBlocks, node.namesystem.getFilesAndDirectoriesTotal(), node.sessionId, node.startupConf, node.confg); } InjectionHandler.processEvent(InjectionEvent.AVATARNODE_SHUTDOWN_COMPLETE); } catch (Exception e) { LOG.error("Failover: shutdownAvatar() failed", e); } finally { LOG.info("Failover: Shutdown thread for " + node.currentAvatar + " DONE."); } } } /** * Return true if the shared journal is active, or if the number * of active journals is equal to the number of configured journals. * Throw IOException otherwise. */ private void verifyEditStreams() throws IOException { // we check if the shared stream is still available if (getFSImage().getEditLog().isSharedJournalAvailable() && InjectionHandler.trueCondition(InjectionEvent.AVATARNODE_CHECKEDITSTREAMS)) { return; } // for sanity check if the number of available journals // is equal to the number of configured ones int expectedEditStreams = NNStorageConfiguration.getNamespaceEditsDirs(confg).size(); int actualEditStreams = this.namesystem.getFSImage().getEditLog().getNumberOfAvailableJournals(); if (expectedEditStreams == actualEditStreams && InjectionHandler.trueCondition(InjectionEvent.AVATARNODE_CHECKEDITSTREAMS)) { return; } String msg = "Failover: Cannot proceed - shared journal is not available. " + "Number of required edit streams: " + expectedEditStreams + " current number: " + actualEditStreams; LOG.fatal(msg); throw new IOException(msg); } /** * Shuts down the avatar node * @param synchronous - should the function wait for the shutdown to complete * @throws IOException */ public synchronized void shutdown(boolean synchronous) throws IOException { LOG.info("Failover: Asynchronous shutdown for: " + currentAvatar); // check permissions before any other actions super.namesystem.checkSuperuserPrivilege(); if (runInfo.shutdown) { LOG.info("Failover: Node already shut down"); return; } // check edit streams // if this fails, we still have a chance to fix it // and shutdown again verifyEditStreams(); runInfo.shutdown = true; Thread shutdownThread = new ShutdownAvatarThread(this); shutdownThread.setName("ShutDown thread for : " + serverAddress); shutdownThread.setDaemon(false); shutdownThread.start(); if (synchronous) { LOG.info("Failover: Waiting for shutdown to complete"); try { shutdownThread.join(); } catch (InterruptedException ie) { throw new IOException(ie); } } } @Override public void shutdownAvatar() throws IOException { shutdown(false); } /** * Used only for testing. */ public Standby getStandby() throws IOException { if (currentAvatar != Avatar.STANDBY) { throw new IOException("This is not the standby avatar"); } return standby; } public long getSessionId() throws IOException { if (currentAvatar != Avatar.ACTIVE) { throw new IOException("This is not the primary avatar"); } return this.sessionId; } /** * Used only for testing. */ public void quiesceStandby(long txId) throws IOException { if (currentAvatar != Avatar.STANDBY) { throw new IOException("This is not the standby avatar"); } standby.quiesce(txId); } public void shutdownStandby() { standby.shutdown(); if (server != null) { // shutdown the AvatarNode LOG.info("Stopping avatarnode rpcserver."); server.stop(); try { server.join(); } catch (InterruptedException ie) { //eat it up } } if (cleaner != null) { // Shut down the cleaner thread as it will keep // the process from shutting down cleaner.stop(); cleanerThread.interrupt(); try { cleanerThread.join(); } catch (InterruptedException iex) { Thread.currentThread().interrupt(); } } } /** * Stops all RPC threads and ensures that all RPC handlers have exited. * Stops all communication to the namenode. */ protected void stopRPC(boolean interruptClientHandlers) throws IOException { try { // stop avatardatanode server stopRPCInternal(server, "avatardatanode", interruptClientHandlers); // stop namenode rpc (client, datanode) super.stopRPC(interruptClientHandlers); // wait for avatardatanode rpc stopWaitRPCInternal(server, "avatardatanode"); } catch (InterruptedException ex) { throw new IOException("stopRPC() interrupted", ex); } } private void verifyTransactionIds(ZookeeperTxId zkTxId) throws IOException { // TODO for unit test it can happen than rollEditLog happens after // obtaining the txid so we might have a difference of 2 !!! long zkLastTxId = zkTxId.getTransactionId(); long zkTotalBlocks = zkTxId.getTotalBlocks(); long zkTotalInodes = zkTxId.getTotalInodes(); // local values long lastTxId = super.getLastWrittenTxId(); long totalBlocks = super.namesystem.getBlocksTotal(); long totalINodes = super.namesystem.getFilesAndDirectoriesTotal(); // Verify transacation ids. if (lastTxId < 0 || zkLastTxId < 0) { throw new StandbyStateException("Invalid transacation ids, txid in NameNode : " + lastTxId + " txid in Zookeeper : " + zkLastTxId); } else if (lastTxId != zkLastTxId) { throw new StandbyStateException("The transacation id in the namenode : " + lastTxId + " does not match the transaction id in zookeeper : " + zkLastTxId + formatErrorMessage(lastTxId, zkLastTxId, "transactions")); } else if (zkTotalBlocks != totalBlocks) { throw new StandbyStateException( "Total blocks in ZK : " + zkTotalBlocks + " don't match up with total blocks on Standby : " + totalBlocks + formatErrorMessage(totalBlocks, zkTotalBlocks, "blocks")); } else if (zkTotalInodes != totalINodes) { throw new StandbyStateException( "Total inodes in ZK : " + zkTotalInodes + " don't match up with total inodes on Standby : " + totalINodes + formatErrorMessage(totalINodes, zkTotalInodes, "inodes")); } } private String formatErrorMessage(long localNumber, long remoteNumber, String what) { long diff = localNumber - remoteNumber; return " Standby has " + Math.abs(diff) + " " + ((diff > 0) ? "more " : "fewer ") + what + "."; } private static File buildSnapshotFilePath(String pathDir) { return new File(pathDir, Path.SEPARATOR + FAILOVER_SNAPSHOT_FILE); } private File getSnapshotFile(Configuration conf, boolean remote) throws IOException { URI sharedEditsDirectory = remote ? getRemoteSharedEditsURI(confg) : getLocalSharedEditsURI(confg); if (!isFile(sharedEditsDirectory)) { String failoverDataDir = conf.get("dfs.avatarnode.failover.test.data.dir"); if (failoverDataDir != null) { return buildSnapshotFilePath(failoverDataDir); } return null; } return buildSnapshotFilePath(sharedEditsDirectory.getPath()); } private void writeFailoverTestData(String fsck) throws IOException { if (!enableTestFramework) { LOG.info("Failover: Test framework - disabled"); return; } File snapshotFile = getSnapshotFile(confg, true); if (snapshotFile == null) return; float samplePercent = confg.getFloat("dfs.avatarnode.failover.sample.percent", 0.05f); LOG.info("Failover: Test framework - using " + (100.0 * samplePercent) + " % sample size"); List<FileStatusExtended> stat = super.getRandomFilesSample(samplePercent); AvatarFailoverSnapshot snapshot = new AvatarFailoverSnapshot(super.namesystem.getOpenFiles(), stat); DataOutputStream out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(snapshotFile))); try { snapshot.write(out); out.writeBoolean(enableTestFrameworkFsck); if (enableTestFrameworkFsck) { Text.writeString(out, fsck); } } finally { out.close(); } LOG.info("Failover: Test framework - saved snapshot file : " + snapshotFile); } private void verifySnapshotSampledFile(FileStatusExtended file) throws IOException { FileStatusExtended stat = super.namesystem.getFileInfoExtended(file.getPath().toString()); if (!stat.equals(file)) { throw new IOException("Information for file : " + file.getPath() + " does not match with information on snapshot file, expected : " + file + ", actual : " + stat); } } private void verifyOpenFiles(OpenFilesInfo openFilesInfo) throws IOException { if (openFilesInfo.getGenStamp() != super.namesystem.getGenerationStamp()) { throw new IOException("GS on snapshot file doesn't match with GS on node : " + openFilesInfo.getGenStamp() + ", " + super.namesystem.getGenerationStamp()); } for (FileStatusExtended stat : openFilesInfo.getOpenFiles()) { verifySnapshotSampledFile(stat); } } private void verifyFailoverTestData() throws IOException { if (!enableTestFramework) { LOG.info("Failover: Test framework - disabled"); return; } String fsck = ""; LOG.info("Failover: Test framework - verification - starting..."); AvatarFailoverSnapshot snapshot = new AvatarFailoverSnapshot(); File snapshotFile = getSnapshotFile(confg, false); if (snapshotFile == null) return; DataInputStream in = new DataInputStream(new BufferedInputStream(new FileInputStream(snapshotFile))); try { snapshot.readFields(in); if (in.readBoolean()) { LOG.info("Failover: Test framework - found fsck data"); fsck = Text.readString(in); } } finally { in.close(); } LOG.info("Failover: Test framework - verifying open files: found " + snapshot.getOpenFilesInfo().getOpenFiles().size() + " files in the test snapshot"); verifyOpenFiles(snapshot.getOpenFilesInfo()); LOG.info("Failover: Test framework - verifying closed files: found " + snapshot.getSampledFiles().size() + " files in the test snapshot"); for (FileStatusExtended stat : snapshot.getSampledFiles()) { verifySnapshotSampledFile(stat); } LOG.info("Failover: Test framework - verification - succeeded"); this.oldPrimaryFsck = fsck; } protected String runFailoverFsck() throws IOException { Map<String, String[]> pmap = new HashMap<String, String[]>(); pmap.put("path", new String[] { "/" }); // run fsck StringWriter stringWriter = new StringWriter(); NamenodeFsck fscker = new NamenodeFsck(confg, this, pmap, new PrintWriter(stringWriter)); fscker.fsck(); return stringWriter.toString(); } /** * Clean failover-outstanding datanodes. * Used for fast failover, once the overreplication processing starts. */ void clearOutstandingNodes() { if (standbySafeMode != null) { standbySafeMode.removeOutStandingDatanodes(false); } standbySafeMode = null; } @Override public synchronized void performFailover() throws IOException { // check permissions before any other actions super.namesystem.checkSuperuserPrivilege(); failoverState = FailoverState.PERFORM_FAILOVER; enforceStandby("Cannot perform failover on active"); cleaner.stop(); cleanerThread.interrupt(); try { cleanerThread.join(); } catch (InterruptedException iex) { Thread.currentThread().interrupt(); } // change the value to the one for the primary int maxStandbyBufferedTransactions = confg.getInt("dfs.max.buffered.transactions", HdfsConstants.DEFAULT_MAX_BUFFERED_TRANSACTIONS); FSEditLog.setMaxBufferedTransactions(maxStandbyBufferedTransactions); // Clear up deletion and replication queues. clearInvalidateAndReplicationQueues(); // if the log was closed by ingestion, re-open it if (!getFSImage().getEditLog().isOpen()) getFSImage().getEditLog().open(); LOG.info("Failover: Triggering safemode failover"); standbySafeMode.triggerFailover(); sessionId = AvatarNodeZkUtil.writeToZooKeeperAfterFailover(startupConf, confg); LOG.info("Failover: Changed avatar from " + currentAvatar + " to " + Avatar.ACTIVE); if (enableTestFramework && enableTestFrameworkFsck) { if (!failoverFsck.equals(oldPrimaryFsck)) { LOG.warn("Failover: FSCK on old primary and new primary do not match"); LOG.info("----- FSCK ----- OLD BEGIN"); LOG.info("Failover: Old primary fsck: \n " + oldPrimaryFsck + "\n"); LOG.info("----- FSCK ----- NEW BEGIN"); LOG.info("Failover: New primary fsck: \n " + failoverFsck + "\n"); LOG.info("----- FSCK ----- END"); } else { LOG.info("Failover: Verified fsck."); } } currentAvatar = Avatar.ACTIVE; confg.setClass("dfs.safemode.impl", NameNodeSafeModeInfo.class, SafeModeInfo.class); } @Override public synchronized void quiesceForFailover(boolean noverification) throws IOException { // check permissions before any other actions super.namesystem.checkSuperuserPrivilege(); failoverState = FailoverState.START_FAILOVER; enforceStandby("Cannot quiesce primary"); try { // Check to see if the primary is somehow checkpointing itself. If so, // then // refuse to switch to active mode. This check is not foolproof but is a // defensive mechanism to prevent administrator errors. ZookeeperTxId zkTxId = AvatarNodeZkUtil.checkZooKeeperBeforeFailover(startupConf, startupConf, noverification); if (!noverification) { if (zkTxId == null) { throw new IOException("Could not receive last transaction id from zookeeper"); } standby.quiesce(zkTxId.getTransactionId()); } else { standby.quiesce(FSEditLogLoader.TXID_IGNORE); } if (!noverification) { verifyTransactionIds(zkTxId); verifyFailoverTestData(); } failoverState = FailoverState.AWAIT_FAILOVER; } catch (StandbyStateException se) { failoverState = FailoverState.AWAIT_FAILOVER; throw se; } finally { if (failoverState != FailoverState.AWAIT_FAILOVER) { failoverState = FailoverState.FAILED_FAILOVER; } } } /** * @deprecated */ @Override public synchronized void setAvatar(Avatar avatar) throws IOException { setAvatar(avatar, false); } /** * @deprecated */ @Override public synchronized void setAvatar(Avatar avatar, boolean force) throws IOException { try { if (avatar == currentAvatar) { LOG.info("Failover: Trying to change avatar to " + avatar + " but am already in that state."); return; } quiesceForFailover(force); performFailover(); } catch (IOException e) { LOG.fatal("Exception when setting avatar", e); throw e; } } /* * As the AvatarNode is running in Standby mode it fills up * invalidates queues for each datanode with blocks it * assumes have to be deleted. This information is not * entirely accurate and fills up memory as well as leads * to dataloss since those queues are flushed to the datanodes * on failover and valid blocks may be deleted. * * To help prevent filling up the memory we clear these queues * periodically. And we do a final cleanup jsut before switching * to primary. * * Also, we make sure that the replication queues are cleaned * periodically. They are never processed at standby, and grow * indefinitely. */ private class Cleaner implements Runnable { volatile boolean running = true; @Override public void run() { LOG.info("Starting Standby Cleaner thread"); while (running) { clearInvalidateAndReplicationQueues(); try { Thread.sleep(INVALIDATES_CLEANUP_INTERVAL); } catch (InterruptedException iex) { if (running == false) return; Thread.currentThread().interrupt(); } } } public void stop() { running = false; } } private void clearInvalidateAndReplicationQueues() { try { LOG.info("Standby Cleaner: cleaning queues"); if (!namesystem.isInSafeMode()) { throw new IOException("Avatar is not in safemode"); } DatanodeInfo[] nodes; super.namesystem.readLock(); try { nodes = super.namesystem.getDatanodes(DatanodeReportType.ALL); } finally { super.namesystem.readUnlock(); } super.namesystem.writeLock(); try { for (DatanodeInfo node : nodes) { super.namesystem.removeFromInvalidates(node.getStorageID()); } } finally { super.namesystem.writeUnlock(); } super.namesystem.clearReplicationQueues(); } catch (Exception e) { metrics.numCleanerThreadExceptions.inc(); LOG.error("Standby Cleaner : exception when cleaning " + "replication queues", e); } } private boolean ignoreDatanodes() { boolean ignore = currentAvatar == Avatar.STANDBY && (standby == null || standby.fellBehind() || InjectionHandler.falseCondition(InjectionEvent.STANDBY_FELL_BEHIND)); metrics.ignoreDataNodes.set(ignore ? 1 : 0); return ignore; } @Override public void primaryCleared(DatanodeRegistration registration) { LOG.info("Received primaryCleared() from : " + registration); if (standbySafeMode != null) { standbySafeMode.reportPrimaryCleared(registration); } } @Override public DatanodeRegistration register(DatanodeRegistration nodeReg) throws IOException { DatanodeRegistration reg = super.register(nodeReg); if (standbySafeMode != null) { standbySafeMode.reportRegister(nodeReg); } return reg; } public DatanodeCommand[] sendHeartbeatNew(DatanodeRegistration registration, long capacity, long dfsUsed, long remaining, long namespaceUsed, int xmitsInProgress, int xceiverCount) throws IOException { DatanodeCommand[] cmds = super.sendHeartbeat(registration, capacity, dfsUsed, remaining, namespaceUsed, xmitsInProgress, xceiverCount); if (standbySafeMode != null && standbySafeMode.reportHeartBeat(registration)) { LOG.info("Sending Clear Primary command to : " + registration); cmds = addCommand(cmds, AvatarDatanodeCommand.CLEARPRIMARY); } else if (ignoreDatanodes()) { cmds = addCommand(cmds, AvatarDatanodeCommand.BACKOFF); } else if (standbySafeMode != null && standbySafeMode.getPrepareFailover()) { cmds = addCommand(cmds, AvatarDatanodeCommand.PREPAREFAILOVER); } return cmds; } private DatanodeCommand[] addCommand(DatanodeCommand[] cmds, DatanodeCommand toAdd) { if (cmds == null) { return new DatanodeCommand[] { toAdd }; } else { DatanodeCommand[] newCmds = Arrays.copyOf(cmds, cmds.length + 1); newCmds[cmds.length] = toAdd; return newCmds; } } @Override /** * Determines whether or not the datanode should retry blocks if they are * not present in the blocks map. */ public boolean shouldRetryAbsentBlocks() { return (currentAvatar == Avatar.STANDBY); } @Override /** * Determines whether or not the given block should be retried by the datanode * if its not present in the blocksMap. */ public boolean shouldRetryAbsentBlock(Block block, Block storedBlock) { // If this block does not belong to anyfile and its GS // is no less than the avatar node's GS, // AvatarNode may not consume the file/block creation edit log yet, // so adding it to the retry list. return (currentAvatar == Avatar.STANDBY && (!namesystem.getPersistBlocks() || block.getGenerationStamp() >= namesystem.getGenerationStamp()) && (storedBlock == null || block.getGenerationStamp() > storedBlock.getGenerationStamp())); } public DatanodeCommand blockReportNew(DatanodeRegistration nodeReg, BlockReport rep) throws IOException { if (runInfo.shutdown || !runInfo.isRunning) { return null; } if (ignoreDatanodes()) { LOG.info("Standby fell behind. Telling " + nodeReg.toString() + " to back off"); // Do not process block reports yet as the ingest thread is catching up return AvatarDatanodeCommand.BACKOFF; } if (currentAvatar == Avatar.STANDBY) { Collection<Block> failed = super.blockReportWithRetries(nodeReg, rep); // standby should send only DNA_RETRY BlockCommand bCmd = new BlockCommand(DatanodeProtocols.DNA_RETRY, failed.toArray(new Block[failed.size()])); return bCmd; } else { // only the primary can send DNA_FINALIZE return super.blockReport(nodeReg, rep); } } private void updateIBRMetrics(int reported, int retried) { if (metrics != null) { metrics.numReportedBlocks.inc(reported); metrics.numRetryBlocks.inc(retried); } } /** * @inheritDoc */ public Block[] blockReceivedAndDeletedNew(DatanodeRegistration nodeReg, Block blocksReceivedAndDeleted[]) throws IOException { if (runInfo.shutdown || !runInfo.isRunning) { // Do not attempt to process blocks when // the namenode is not running return new ReceivedBlockInfo[0]; } if (ignoreDatanodes()) { LOG.info("Standby fell behind. Telling " + nodeReg.toString() + " to retry incremental block report of " + blocksReceivedAndDeleted.length + " blocks later."); metrics.numIgnoredDatanodes.inc(); return blocksReceivedAndDeleted; } List<Block> failed = new ArrayList<Block>(); HashSet<Long> failedIds; if (currentAvatar == Avatar.STANDBY) { failedIds = new HashSet<Long>(); namesystem.writeLock(); try { for (int index = 0; index < blocksReceivedAndDeleted.length; index++) { Block blockRD = blocksReceivedAndDeleted[index]; if (failedIds.contains(blockRD.getBlockId())) { // check if there was no other blocking failed request blocksReceivedAndDeleted[index] = null; failed.add(blockRD); continue; } BlockInfo storedBlock = namesystem.blocksMap.getStoredBlock(blockRD); if (!DFSUtil.isDeleted(blockRD) && shouldRetryAbsentBlock(blockRD, storedBlock)) { // If this block does not belong to anyfile and its GS // is no less than the avatar node's GS, // AvatarNode may not consume the file/block creation edit log yet, // so adding it to the failed list. // - do not process any requestes for blocks with the same block id // (also add them to the failed list. // - do not block other requests blocksReceivedAndDeleted[index] = null; failed.add(blockRD); failedIds.add(blockRD.getBlockId()); } } } finally { namesystem.writeUnlock(); if (!failed.isEmpty()) { LOG.info("*BLOCK* NameNode.blockReceivedAndDeleted: " + "from " + nodeReg.getName() + " has to retry " + failed.size() + " blocks."); } for (Block blockRD : failed) { LOG.info("blockReceivedDeleted " + (DFSUtil.isDeleted(blockRD) ? "DELETED" : "RECEIVED") + " request received for " + blockRD + " on " + nodeReg.getName() + " size " + blockRD.getNumBytes() + " But it does not belong to any file." + " Retry later."); } } } super.blockReceivedAndDeleted(nodeReg, blocksReceivedAndDeleted); updateIBRMetrics(blocksReceivedAndDeleted.length, failed.size()); return failed.toArray(new Block[failed.size()]); } /** * @inheritDoc */ public long[] blockReceivedAndDeletedNew(DatanodeRegistration nodeReg, IncrementalBlockReport receivedAndDeletedBlocks) throws IOException { InjectionHandler.processEvent(InjectionEvent.AVATARNODE_BLOCKRECEIVED_AND_DELETED_NEW); long[] failedMap = null; if (runInfo.shutdown || !runInfo.isRunning) { // Do not attempt to process blocks when // the namenode is not running if (currentAvatar == Avatar.STANDBY) { return new long[0]; } else { return null; } } HashSet<Long> failedIds; if (currentAvatar == Avatar.STANDBY) { int noAck = receivedAndDeletedBlocks.getLength(); // retry all block if the standby is behind consuming edits if (ignoreDatanodes()) { LOG.info("Standby fell behind. Telling " + nodeReg.toString() + " to retry incremental block report of " + noAck + " blocks later."); failedMap = LightWeightBitSet.getBitSet(noAck); for (int i = 0; i < noAck; i++) LightWeightBitSet.set(failedMap, i); metrics.numIgnoredDatanodes.inc(); return failedMap; } Block blockRD = new Block(); failedIds = new HashSet<Long>(); failedMap = LightWeightBitSet.getBitSet(noAck); namesystem.writeLock(); try { receivedAndDeletedBlocks.resetIterator(); for (int currentBlock = 0; currentBlock < noAck; currentBlock++) { receivedAndDeletedBlocks.getNext(blockRD); if (failedIds.contains(blockRD.getBlockId())) { // check if there was no other blocking failed request blockRD.setNumBytes(BlockFlags.IGNORE); receivedAndDeletedBlocks.setBlock(blockRD, currentBlock); LightWeightBitSet.set(failedMap, currentBlock); continue; } BlockInfo storedBlock = namesystem.blocksMap.getStoredBlock(blockRD); if ((!DFSUtil.isDeleted(blockRD) && shouldRetryAbsentBlock(blockRD, storedBlock))) { // If this block does not belong to anyfile and its GS // is no less than the avatar node's GS, // AvatarNode may not consume the file/block creation edit log yet, // so adding it to the failed list. Also, if the stored block GS is // less than the reported GS then we need to retry the block. // - do not process any requestes for blocks with the same block id // (also add them to the failed list. // - do not block other requests blockRD.setNumBytes(BlockFlags.IGNORE); receivedAndDeletedBlocks.setBlock(blockRD, currentBlock); LightWeightBitSet.set(failedMap, currentBlock); failedIds.add(blockRD.getBlockId()); } } } finally { namesystem.writeUnlock(); if (failedMap != null && LightWeightBitSet.cardinality(failedMap) != 0) { LOG.info("*BLOCK* NameNode.blockReceivedAndDeleted: " + "from " + nodeReg.getName() + " has to retry " + LightWeightBitSet.cardinality(failedMap) + " blocks."); } receivedAndDeletedBlocks.resetIterator(); for (int currentBlock = 0; currentBlock < noAck; currentBlock++) { receivedAndDeletedBlocks.getNext(blockRD); if (!LightWeightBitSet.get(failedMap, currentBlock)) continue; LOG.info("blockReceivedDeleted " + (DFSUtil.isDeleted(blockRD) ? "DELETED" : "RECEIVED") + " request received for " + blockRD + " on " + nodeReg.getName() + " size " + blockRD.getNumBytes() + " But it does not belong to any file." + " Retry later."); } } } super.blockReceivedAndDeleted(nodeReg, receivedAndDeletedBlocks); updateIBRMetrics(receivedAndDeletedBlocks.getLength(), failedMap != null ? LightWeightBitSet.cardinality(failedMap) : 0); return failedMap; } /** * Roll the edit log. */ public CheckpointSignature rollEditLog() throws IOException { enforceActive("Cannot roll edit log on standby"); verifyCheckpointerAddress(); return super.rollEditLog(); } /** * Roll the edit log manually. */ @Override public void rollEditLogAdmin() throws IOException { enforceActive("Cannot roll edit log on standby"); // no verification of the checkpointer address since this is dfsadmin call super.rollEditLog(); } /** * Roll the image */ public void rollFsImage(CheckpointSignature newImageSignature) throws IOException { enforceActive("Cannot roll image on standby"); verifyCheckpointerAddress(); super.rollFsImage(newImageSignature); } @Override public void saveNamespace() throws IOException { this.saveNamespace(false, false); } @Override public void saveNamespace(boolean force, boolean uncompressed) throws IOException { // for manually triggered SN, reset earlier cancellations namesystem.clearCancelSaveNamespace(); if (currentAvatar == Avatar.ACTIVE) { // regular saving process super.saveNamespace(force, uncompressed); } else /* STANDBY */ { // "force" is not needed for standby since it's always // in safemode standby.triggerCheckpoint(uncompressed); } } /** * @inheritDoc */ @Override public boolean setSafeMode(SafeModeAction action) throws IOException { if (action == SafeModeAction.SAFEMODE_PREP_FAILOVER) { if (currentAvatar == Avatar.ACTIVE) { // for now, we do not take any actions } else /* STANDBY */ { if (standby != null) { standby.disableCheckpoint(); namesystem.cancelSaveNamespace("Prepare for failover"); } if (standbySafeMode != null) { // inform the safemode to prepare failover standbySafeMode.setPrepareFailover(true); } } return namesystem.isInSafeMode(); } return super.setSafeMode(action); } void enforceActive(String msg) throws IOException { if (currentAvatar == Avatar.STANDBY) { throw new IOException(msg); } } private void enforceStandby(String msg) throws IOException { if (currentAvatar != Avatar.STANDBY) { throw new IOException(msg); } } /** * Register standby with this primary */ @Override public int register() throws IOException { enforceActive("Standby can only register with active namenode"); verifyCheckpointerAddress(); return DataTransferProtocol.DATA_TRANSFER_VERSION; } private void verifyCheckpointerAddress() throws IOException { InetAddress configuredRemoteAddress = getRemoteNamenodeAddress(getConf(), instance).getAddress(); validateCheckpointerAddress(configuredRemoteAddress); } /** * Returns the hostname:port for the AvatarNode. The default * port for the AvatarNode is (the client RPC port * of the underlying namenode + 1) */ public static InetSocketAddress getAddress(Configuration conf) { InetSocketAddress u = NameNode.getClientProtocolAddress(conf); int port = conf.getInt(AvatarNode.DFS_AVATARNODE_PORT_KEY, u.getPort() + 1); return new InetSocketAddress(u.getAddress(), port); } /** * Help message for a user */ private static void printUsage() { System.err.println("Usage: java AvatarNode [" + StartupOption.STANDBY.getName() + "] | [" + StartupOption.NODEZERO.getName() + "] | [" + StartupOption.NODEONE.getName() + "] | [" + StartupOption.FORMAT.getName() + "] | [" + StartupOption.UPGRADE.getName() + "] | [" + StartupOption.ROLLBACK.getName() + "] | [" + StartupOption.FINALIZE.getName() + "] | [" + StartupOption.IMPORT.getName() + "]"); } /** * validates command line arguments */ static void validateStartupOptions(StartupInfo startInfo) throws IOException { // sync cannot be specified along with format or finalize if (startInfo.isStandby) { if (startInfo.startOpt == StartupOption.FORMAT || startInfo.startOpt == StartupOption.FINALIZE || startInfo.startOpt == StartupOption.ROLLBACK || startInfo.startOpt == StartupOption.UPGRADE) { throw new IOException( "Standby avatar node cannot be started with " + startInfo.startOpt + " option."); } } } static class StartupInfo { StartupOption startOpt; InstanceId instance; boolean isStandby; String serviceName; boolean forceStartup; public StartupInfo(StartupOption startOpt, InstanceId instance, boolean isStandby, String serviceName, boolean forceStartup) { this.startOpt = startOpt; this.instance = instance; this.isStandby = isStandby; this.serviceName = serviceName; this.forceStartup = forceStartup; } } /** * Analyze the command line options */ private static StartupInfo parseArguments(String args[]) { InstanceId instance = InstanceId.NODEZERO; StartupOption startOpt = StartupOption.REGULAR; boolean isStandby = false; String serviceName = null; boolean force = false; int argsLen = (args == null) ? 0 : args.length; for (int i = 0; i < argsLen; i++) { String cmd = args[i]; if (StartupOption.SERVICE.getName().equalsIgnoreCase(cmd)) { if (++i < argsLen) { serviceName = args[i]; } else { return null; } } else if (StartupOption.STANDBY.getName().equalsIgnoreCase(cmd)) { isStandby = true; } else if (StartupOption.NODEZERO.getName().equalsIgnoreCase(cmd)) { instance = InstanceId.NODEZERO; } else if (StartupOption.NODEONE.getName().equalsIgnoreCase(cmd)) { instance = InstanceId.NODEONE; } else if (StartupOption.FORMAT.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.FORMAT; } else if (StartupOption.FORMATFORCE.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.FORMATFORCE; } else if (StartupOption.REGULAR.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.REGULAR; } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.UPGRADE; } else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.ROLLBACK; } else if (StartupOption.FINALIZE.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.FINALIZE; } else if (StartupOption.IMPORT.getName().equalsIgnoreCase(cmd)) { startOpt = StartupOption.IMPORT; } else if (StartupOption.FORCE.getName().equalsIgnoreCase(cmd)) { force = true; } else { return null; } } return new StartupInfo(startOpt, instance, isStandby, serviceName, force); } /** * Records the startup command in the configuration */ private static void setStartupOption(Configuration conf, StartupOption opt) { conf.set("dfs.namenode.startup", opt.toString()); } public static AvatarNode createAvatarNode(String argv[], Configuration conf) throws IOException { return createAvatarNode(argv, conf, new RunInfo()); } /** * HDFS federation configuration that is specific to a name service. * This keys are suffixed with nameserviceId in the configuration. For example, * "dfs.namenode.rpc-address.nameservice1".</li> * </ol> * * Following are nameservice specific keys. */ final private static String DFS_AVATARNODE_PORT_KEY = "dfs.avatarnode.port"; final private static String DFS_SHARED_NAME_DIR0_KEY = "dfs.name.dir.shared0"; final private static String DFS_SHARED_NAME_DIR1_KEY = "dfs.name.dir.shared1"; final public static String DFS_SHARED_EDITS_DIR0_KEY = "dfs.name.edits.dir.shared0"; final public static String DFS_SHARED_EDITS_DIR1_KEY = "dfs.name.edits.dir.shared1"; final private static String ZERO = "0"; final private static String ONE = "1"; final public static String DFS_NAMENODE_RPC_ADDRESS0_KEY = DFS_NAMENODE_RPC_ADDRESS_KEY + ZERO; final public static String DFS_NAMENODE_RPC_ADDRESS1_KEY = DFS_NAMENODE_RPC_ADDRESS_KEY + ONE; public static final String[] AVATARSERVICE_SPECIFIC_KEYS = { DFS_AVATARNODE_PORT_KEY, DFS_NAMENODE_RPC_ADDRESS0_KEY, DFS_NAMENODE_RPC_ADDRESS1_KEY, DATANODE_PROTOCOL_ADDRESS + ZERO, DATANODE_PROTOCOL_ADDRESS + ONE, DFS_NAMENODE_HTTP_ADDRESS_KEY + ZERO, DFS_NAMENODE_HTTP_ADDRESS_KEY + ONE, DFS_SHARED_NAME_DIR0_KEY, DFS_SHARED_NAME_DIR1_KEY, DFS_SHARED_EDITS_DIR0_KEY, DFS_SHARED_EDITS_DIR1_KEY, }; /** * In federation configuration is set for a set of * avartanodes, namenodes etc, which are * grouped under a logical nameservice ID. The configuration keys specific * to them have suffix set to configured nameserviceId. * * This method copies the value from specific key of format key.nameserviceId * to key, to set up the generic configuration. Once this is done, only * generic version of the configuration is read in rest of the code, for * backward compatibility and simpler code changes. * * @param conf * Configuration object to lookup specific key and to set the value * to the key passed. Note the conf object is modified * @see DFSUtil#setGenericConf(Configuration, String, String...) */ public static void initializeGenericKeys(Configuration conf, String serviceKey) { if ((serviceKey == null) || serviceKey.isEmpty()) { return; } NameNode.initializeGenericKeys(conf, serviceKey); DFSUtil.setGenericConf(conf, serviceKey, AVATARSERVICE_SPECIFIC_KEYS); // adjust meta directory names for this service adjustMetaDirectoryNames(conf, serviceKey); } /** Append service name to each avatar meta directory name * * @param conf configuration of NameNode * @param serviceKey the non-empty name of the name node service */ public static void adjustMetaDirectoryNames(Configuration conf, String serviceKey) { adjustMetaDirectoryName(conf, DFS_SHARED_NAME_DIR0_KEY, serviceKey); adjustMetaDirectoryName(conf, DFS_SHARED_NAME_DIR1_KEY, serviceKey); adjustMetaDirectoryName(conf, DFS_SHARED_EDITS_DIR0_KEY, serviceKey); adjustMetaDirectoryName(conf, DFS_SHARED_EDITS_DIR1_KEY, serviceKey); } @Override public void reconfigurePropertyImpl(String property, String newVal) throws ReconfigurationException { String expectation = ""; switch (instance) { case NODEZERO: expectation = DFS_NAMENODE_RPC_ADDRESS1_KEY; break; case NODEONE: expectation = DFS_NAMENODE_RPC_ADDRESS0_KEY; break; } if (property.equals(expectation)) { getConf().set(property, newVal); return; } super.reconfigurePropertyImpl(property, newVal); } @Override public List<String> getReconfigurableProperties() { List<String> allProps = super.getReconfigurableProperties(); switch (instance) { case NODEONE: allProps.add(DFS_NAMENODE_RPC_ADDRESS0_KEY); break; case NODEZERO: allProps.add(DFS_NAMENODE_RPC_ADDRESS1_KEY); break; } return allProps; } /** * Tries to bind to the address specified in ZooKeeper, this will always fail * if the primary is alive either on the same machine or on a remote machine. */ private static void isPrimaryAlive(String zkRegistry) throws IOException { String parts[] = zkRegistry.split(":"); if (parts.length != 2) { throw new IllegalArgumentException("Invalid Address : " + zkRegistry); } String host = parts[0]; int port = Integer.parseInt(parts[1]); InetSocketAddress clientSocket = new InetSocketAddress(host, port); ServerSocket socket = new ServerSocket(); socket.bind(clientSocket); socket.close(); } private static void failStartup(String message) throws IOException { LOG.error(message); throw new IOException(message); } private static String getWildcardDir(String instance, Configuration conf, String dirKey) { String dir = conf.get(dirKey); dir = dir.replaceAll(FSConstants.DFS_NAMENODE_NAME_DIR_WILDCARD, instance); return dir; } /** * When avatarone and avatarzero both write to the filer in the case where the * filer is not the shared directory (when we use QJM). We need both the nodes * to write to different directories. For this purpose we configure * dfs.name.dir and dfs.name.edits.dir as follows * * <property> * <name>dfs.name.dir</name> * <value>/hadoop/<cluster>/,/mnt/fsimage/<cluster>/%</value> * </property> * * Then depending upon the instance we replace '%' with zero or one in this * function to ensure both nodes write to different locations. */ private static void processNameDirectories(Configuration conf, InstanceId instanceId) { if (instanceId == InstanceId.NODEONE || instanceId == InstanceId.NODEZERO) { String instance = (instanceId == InstanceId.NODEZERO) ? "zero" : "one"; // Edits directory. String editDirs = getWildcardDir(instance, conf, FSConstants.DFS_NAMENODE_EDITS_DIR_KEY); conf.set(FSConstants.DFS_NAMENODE_EDITS_DIR_KEY, editDirs); // Image directory. String imageDirs = getWildcardDir(instance, conf, FSConstants.DFS_NAMENODE_NAME_DIR_KEY); conf.set(FSConstants.DFS_NAMENODE_NAME_DIR_KEY, imageDirs); } } public static AvatarNode createAvatarNode(String argv[], Configuration conf, RunInfo runInfo) throws IOException { if (conf == null) { conf = new Configuration(); } Configuration startupConf = conf; // save configuration at startup StartupInfo startInfo = parseArguments(argv); StartupOption startOpt = startInfo.startOpt; if (startInfo.instance != null) { processNameDirectories(conf, startInfo.instance); processNameDirectories(startupConf, startInfo.instance); } if (startOpt == null) { printUsage(); return null; } if (!validateServiceName(conf, startInfo.serviceName)) { return null; } initializeGenericKeys(conf, startInfo.serviceName); setStartupOption(conf, startOpt); conf = updateAddressConf(conf, startInfo.instance); NameNode.setupDefaultURI(conf); // sync cannot be specified along with format or finalize validateStartupOptions(startInfo); // We need to check the zookeeper so that the node starting as active // is the one registered with the zookeeper // and if the node is starting as standby there has to be a master // already so that the node doesn't move the log and the image String fsname = startupConf.get(NameNode.DFS_NAMENODE_RPC_ADDRESS_KEY); String actualName = conf.get(NameNode.DFS_NAMENODE_RPC_ADDRESS_KEY); String errorMsg = null; String zkRegistry = AvatarNodeZkUtil.getPrimaryRegistration(startupConf, startupConf, fsname); if (zkRegistry == null) { // The registry is empty. Usually this means failover is in progress // we need to manually fix it before starting primary if (!startInfo.forceStartup) { errorMsg = "A zNode that indicates the primary is empty. " + "AvatarNode can only be started as primary if it " + "is registered as primary with ZooKeeper"; failStartup(errorMsg); } } else { if (!zkRegistry.equalsIgnoreCase(actualName) && !startInfo.isStandby) { errorMsg = "Registration information in ZooKeeper doesn't " + "match the address of this node. AvatarNode can " + "only be started as primary if it is registered as " + "primary with ZooKeeper. zkRegistry = " + zkRegistry + ", actual name = " + actualName; failStartup(errorMsg); } } if (!startInfo.isStandby && !startInfo.forceStartup) { isPrimaryAlive(zkRegistry); } long ssid = 0; // We are the primary avatar, write session Id to ZK. if (!startInfo.isStandby) { ssid = AvatarNodeZkUtil.writeSessionIdToZK(startupConf); } // If sync is requested, then we copy only the fsimage // (and not the transaction logs) from the other node. // If we are NODEONE, then modify the configuration to // set fs.name.dir, fs.default.name and dfs.http.address. // // setup rpc proxy if we are starting as standby NamenodeProtocol primaryNamenode = null; InetSocketAddress nameNodeAddr = null; if (startInfo.isStandby) { nameNodeAddr = getRemoteNamenodeAddress(conf, startInfo.instance); FLOG.info("Connecting to the primary namenode: " + nameNodeAddr); primaryNamenode = (NamenodeProtocol) RPC.waitForProxy(NamenodeProtocol.class, NamenodeProtocol.versionID, nameNodeAddr, conf); // make sure we can talk to this primary int primaryDataTransferVersion = primaryNamenode.register(); // check if we have the same data transfer version as primary InjectionHandler.processEventIO(InjectionEvent.AVATARNODE_RECEIVED_DATA_TRANSFER_VERSION, primaryDataTransferVersion); int standbyDataTransferVersion = DataTransferProtocol.DATA_TRANSFER_VERSION; if (standbyDataTransferVersion != primaryDataTransferVersion && !startInfo.forceStartup) { throw new IncorrectVersionException(primaryDataTransferVersion, "data transfer", standbyDataTransferVersion); } } conf = setupAvatarNodeStorage(conf, startInfo, primaryNamenode); // namenode options. switch (startOpt) { case FORMAT: boolean aborted = format(conf, false, true); System.exit(aborted ? 1 : 0); case FORMATFORCE: aborted = format(conf, true, false); return null; case FINALIZE: aborted = finalize(conf, true); System.exit(aborted ? 1 : 0); default: } // We need to put the Namenode into safemode as soon as it starts up. // There is a race condition, where before the Standby AvatarNode can put // the NameNode into safemode, the NameNode might leave safemode. This could // occur in the case of a start where the FSImage and FSEdits are empty // and hence the NameNode doesn't wait at all in safemode. if (startInfo.isStandby) { conf.setClass("dfs.safemode.impl", StandbySafeMode.class, SafeModeInfo.class); } // set persisting blocks to be true conf.setBoolean("dfs.persist.blocks", true); return new AvatarNode(startupConf, conf, startInfo, runInfo, ssid, nameNodeAddr, primaryNamenode); } private static boolean isFile(URI uri) throws IOException { return uri.getScheme().compareTo(JournalType.FILE.name().toLowerCase()) == 0; } /** * Return the configuration that should be used by this instance of AvatarNode * Copy fsimages from the remote shared device. */ static Configuration setupAvatarNodeStorage(Configuration conf, StartupInfo startInfo, NamenodeProtocol primaryNamenode) throws IOException { // shared loations for image and edits URI img0 = NNStorageConfiguration.getURIKey(conf, "dfs.name.dir.shared0"); URI img1 = NNStorageConfiguration.getURIKey(conf, "dfs.name.dir.shared1"); URI edit0 = NNStorageConfiguration.getURIKey(conf, "dfs.name.edits.dir.shared0"); URI edit1 = NNStorageConfiguration.getURIKey(conf, "dfs.name.edits.dir.shared1"); // local locations for image and edits Collection<URI> namedirs = NNStorageConfiguration.getNamespaceDirs(conf, null); Collection<URI> editsdir = NNStorageConfiguration.getNamespaceEditsDirs(conf, null); // validate correctness of the configuration AvatarStorageSetup.validate(conf, namedirs, editsdir, img0, img1, edit0, edit1); FileSystem localFs = FileSystem.getLocal(conf).getRaw(); URI ownSharedImage = null; URI ownSharedEdits = null; // // if we are instance one then copy from primary to secondary // otherwise copy from secondary to primary. // if (startInfo.instance == InstanceId.NODEONE) { ownSharedImage = img1; ownSharedEdits = edit1; } else if (startInfo.instance == InstanceId.NODEZERO) { ownSharedImage = img0; ownSharedEdits = edit0; } // allocate a new configuration and update fs.name.dir approprately // The shared device should be the first in the list. Configuration newconf = new Configuration(conf); AvatarStorageSetup.updateConf(startInfo, newconf, namedirs, img0, img1, "dfs.name.dir"); // update fs.name.edits.dir approprately in the new configuration // The shared device should be the first in the list. AvatarStorageSetup.updateConf(startInfo, newconf, editsdir, edit0, edit1, "dfs.name.edits.dir"); // copy fsimage directory if needed if (startInfo.isStandby) { // do not open edit log at startup newconf.setBoolean("dfs.namenode.openlog", false); // connect to primary String fsName = getRemoteNamenodeHttpName(conf, startInfo.instance); FSImage tempImage = new FSImage(newconf, NNStorageConfiguration.getNamespaceDirs(newconf), NNStorageConfiguration.getNamespaceEditsDirs(newconf), null); // will block until Primary has left the safemode CheckpointSignature cs = getCheckpointSignature(primaryNamenode); long lastCheckpointTxId = cs.mostRecentCheckpointTxId; if (cs.layoutVersion != FSConstants.LAYOUT_VERSION) { throw new IOException("Upgrade for standby is not supported"); } if (isFile(ownSharedImage)) { File destFile = new File(ownSharedImage.getPath()); NNStorageDirectoryRetentionManager.backupFiles(localFs, destFile, conf); } if (isFile(ownSharedEdits)) { File destFile = new File(ownSharedEdits.getPath()); NNStorageDirectoryRetentionManager.backupFiles(localFs, destFile, newconf); } // setup storage NNStorage tempStorage = tempImage.storage; tempStorage.format(); tempStorage.setStorageInfo(cs); tempStorage.writeAll(); tempImage.editLog.transitionNonFileJournals(tempStorage, false, Transition.FORMAT, null); tempImage.transitionNonFileImages(tempStorage, false, Transition.FORMAT); // we need to become the active writer to upload image successfully to // non-file images storage tempImage.editLog.recoverUnclosedStreams(); // Download the image to all storage directories FLOG.info("Downloading image to all storage directories."); MD5Hash digest = downloadImageToStorage(fsName, lastCheckpointTxId, tempImage); List<StorageDirectory> badSds = new ArrayList<StorageDirectory>(); tempStorage.checkpointUploadDone(lastCheckpointTxId, digest); FLOG.info("Downloading image to all storage directories. DONE"); tempImage.saveDigestAndRenameCheckpointImage(lastCheckpointTxId, digest); tempStorage.reportErrorsOnDirectories(badSds, tempImage); tempStorage.close(); tempImage.close(); } return newconf; } private static MD5Hash downloadImageToStorage(String fsName, long lastCheckpointTxId, FSImage tempImage) throws IOException { IOException e = null; for (int i = 0; i < 3; i++) { try { // disable throttling for image download return TransferFsImage.downloadImageToStorage(fsName, lastCheckpointTxId, tempImage, true, true); } catch (SocketTimeoutException ex) { e = ex; LOG.info("Downloading image - socked timeout exception. Will retry..."); try { Thread.sleep(1000); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); break; } } } throw e; } private static CheckpointSignature getCheckpointSignature(NamenodeProtocol primaryNamenode) throws IOException { while (true) { try { return primaryNamenode.getCheckpointSignature(); } catch (RemoteException e) { LOG.info("Active namenode is not available. Standby cannot initialize", e); try { Thread.sleep(1000); } catch (InterruptedException ex) { throw new IOException(ex); } } } } public static Configuration updateAddressConf(Configuration conf, InstanceId instance) { Configuration newconf = new Configuration(conf); // if we are starting as the other namenode, then change the // default URL to make the namenode attach to the appropriate URL if (instance == InstanceId.NODEZERO) { String fs = conf.get("dfs.http.address0"); if (fs != null) { newconf.set("dfs.http.address", fs); } fs = conf.get("dfs.namenode.dn-address0"); if (fs != null) { newconf.set("dfs.namenode.dn-address", fs); } fs = conf.get(AvatarNode.DFS_NAMENODE_RPC_ADDRESS0_KEY); if (fs != null) { newconf.set(AvatarNode.DFS_NAMENODE_RPC_ADDRESS_KEY, fs); newconf.set("fs.default.name0", fs); conf.set("fs.default.name0", fs); } fs = conf.get("fs.default.name0"); if (fs != null) { newconf.set("fs.default.name", fs); } } if (instance == InstanceId.NODEONE) { String fs = conf.get("dfs.http.address1"); if (fs != null) { newconf.set("dfs.http.address", fs); } fs = conf.get("dfs.namenode.dn-address1"); if (fs != null) { newconf.set("dfs.namenode.dn-address", fs); } fs = conf.get(AvatarNode.DFS_NAMENODE_RPC_ADDRESS1_KEY); if (fs != null) { newconf.set(AvatarNode.DFS_NAMENODE_RPC_ADDRESS_KEY, fs); newconf.set("fs.default.name1", fs); conf.set("fs.default.name1", fs); } fs = conf.get("fs.default.name1"); if (fs != null) { newconf.set("fs.default.name", fs); } } return newconf; } /** * Returns the address of the remote namenode */ static InetSocketAddress getRemoteNamenodeAddress(Configuration conf, InstanceId instance) throws IOException { String fs = null; if (instance == InstanceId.NODEZERO) { fs = conf.get(DFS_NAMENODE_RPC_ADDRESS1_KEY); if (fs == null) fs = conf.get("fs.default.name1"); } else if (instance == InstanceId.NODEONE) { fs = conf.get(DFS_NAMENODE_RPC_ADDRESS0_KEY); if (fs == null) fs = conf.get("fs.default.name0"); } else { throw new IOException("Unknown instance " + instance); } if (fs != null) { Configuration newConf = new Configuration(conf); newConf.set(FSConstants.DFS_NAMENODE_RPC_ADDRESS_KEY, fs); conf = newConf; } return NameNode.getClientProtocolAddress(conf); } /** * Returns the name of the http server of the local namenode */ static String getRemoteNamenodeHttpName(Configuration conf, InstanceId instance) throws IOException { if (instance == InstanceId.NODEZERO) { return conf.get("dfs.http.address1"); } else if (instance == InstanceId.NODEONE) { return conf.get("dfs.http.address0"); } else { throw new IOException("Unknown instance " + instance); } } /** * Return the shared edits file of the remote NameNode */ URI getRemoteSharedEditsURI(Configuration conf) throws IOException { return getSharedEditsURI(conf, true); } /** * Return the shared edits file of the local NameNode */ URI getLocalSharedEditsURI(Configuration conf) throws IOException { return getSharedEditsURI(conf, false); } private URI getSharedEditsURI(Configuration conf, boolean remote) throws IOException { URI edits = null; if (instance == InstanceId.NODEZERO) { edits = remote ? NNStorageConfiguration.getURIKey(conf, "dfs.name.edits.dir.shared1") : NNStorageConfiguration.getURIKey(conf, "dfs.name.edits.dir.shared0"); } else if (instance == InstanceId.NODEONE) { edits = remote ? NNStorageConfiguration.getURIKey(conf, "dfs.name.edits.dir.shared0") : NNStorageConfiguration.getURIKey(conf, "dfs.name.edits.dir.shared1"); } else { LOG.info("Instance is invalid. " + instance); throw new IOException("Instance is invalid. " + instance); } return edits; } /** * Returns the starting checkpoint time of this AvatarNode */ long getStartCheckpointTxId() { return startCheckpointTxId; } void setStartCheckpointTxId(long txid) { startCheckpointTxId = txid; } /** * Indicates that the AvatarNode shoudl restart */ void doRestart() { runInfo.doRestart = true; } /** * Current system time. * @return current time in msec. */ static long now() { return System.currentTimeMillis(); } protected Map<NameNodeKey, String> getNameNodeSpecificKeys() { Map<NameNodeKey, String> map = new HashMap<NameNodeKey, String>(); try { map.put(new NameNodeKey("Last applied transaction id", NameNodeKey.BOTH), toStr(getFSImage().getEditLog().getLastWrittenTxId())); if (currentAvatar == Avatar.STANDBY) { map.put(new NameNodeKey("Standby: ignore datanodes", NameNodeKey.STANDBY), toStr(this.ignoreDatanodes())); map.put(new NameNodeKey("Standby: ingest state", NameNodeKey.STANDBY), toStr((standby == null) ? "" : standby.currentIngestState)); map.put(new NameNodeKey("Standby: ingest fell behind", NameNodeKey.STANDBY), toStr((standby == null) ? "" : standby.fellBehind())); map.put(new NameNodeKey("Standby: ingest lag bytes", NameNodeKey.STANDBY), toStr((standby == null) ? 0L : standby.getLagBytes())); map.put(new NameNodeKey("Standby: checkpoint status", NameNodeKey.STANDBY), toStr((standby == null) ? "" : standby.getCheckpointStatus())); map.put(new NameNodeKey("Standby: failover in progress", NameNodeKey.STANDBY), toStr(standbySafeMode.failoverInProgress())); if (standbySafeMode.failoverInProgress()) { map.put(new NameNodeKey("Standby: failover outstanding heartbeats", NameNodeKey.STANDBY), toStr(standbySafeMode.getOutStandingHeartbeats().size())); map.put(new NameNodeKey("Standby: failover outstanding reports", NameNodeKey.STANDBY), toStr(standbySafeMode.getOutStandingReports().size())); } } else { map.put(new NameNodeKey("Last checkpoint txid", NameNodeKey.ACTIVE), toStr(this.getFSImage().storage.getMostRecentCheckpointTxId())); map.put(new NameNodeKey("Last checkpoint time", NameNodeKey.ACTIVE), this.getFSImage().storage.getMostRecentCheckpointTime()); } } catch (Exception e) { // send partial information LOG.error(e.toString()); } return map; } protected boolean getIsPrimary() { return currentAvatar == Avatar.ACTIVE; } private String toStr(Object o) { return o.toString(); } public static class RunInfo { volatile boolean doRestart; volatile boolean shutdown; volatile boolean isRunning; public RunInfo(boolean doRestart, boolean shutdown, boolean isRunning) { this.doRestart = doRestart; this.shutdown = shutdown; this.isRunning = isRunning; } public RunInfo() { this.doRestart = false; this.shutdown = false; this.isRunning = true; } } public InetSocketAddress getNameNodeAddress() { return serverAddress; } public StandbySafeMode getStandbySafeMode() { return this.standbySafeMode; } public InstanceId getInstanceId() { return this.instance; } /** */ public static void main(String argv[]) throws Exception { org.apache.hadoop.hdfs.DnsMonitorSecurityManager.setTheManager(); Exception exception = null; AvatarNode avatarnode = null; RunInfo runInfo = new RunInfo(); do { runInfo.doRestart = false; runInfo.isRunning = true; exception = null; try { StringUtils.startupShutdownMessage(AvatarNode.class, argv, LOG); FastWritableHDFS.init(); FastProtocolHDFS.init(); avatarnode = createAvatarNode(argv, null, runInfo); if (avatarnode != null) { avatarnode.waitForRestart(); } } catch (Throwable e) { LOG.error(StringUtils.stringifyException(e)); if (runInfo.doRestart) { LOG.error("AvatarNode restarting..."); } else { exception = new Exception(StringUtils.stringifyException(e)); } } } while (runInfo.doRestart == true); if (runInfo.shutdown) { avatarnode.stopRPC(true); } if (exception != null) { LOG.fatal("Exception running avatarnode. Shutting down", exception); Runtime.getRuntime().exit(1); } } public boolean isInitDone() { return isInitialized; } }