LOG.info("killing datanode " + name + " / " + lookup); ipcPort = dn.ipcServer.getListenerAddress().getPort(); dn.shutdown(); LOG.info("killed datanode " + name + " / " + lookup); break;
/** 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); }
private DataXceiver(Peer peer, DataNode datanode, DataXceiverServer dataXceiverServer) throws IOException { super(datanode.getTracer()); this.peer = peer; this.dnConf = datanode.getDnConf(); this.socketIn = peer.getInputStream(); this.socketOut = peer.getOutputStream(); this.datanode = datanode; this.dataXceiverServer = dataXceiverServer; this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname; this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(datanode.getConf()); this.smallBufferSize = DFSUtilClient.getSmallBufferSize(datanode.getConf()); remoteAddress = peer.getRemoteAddressString(); final int colonIdx = remoteAddress.indexOf(':'); remoteAddressWithoutPort = (colonIdx < 0) ? remoteAddress : remoteAddress.substring(0, colonIdx); localAddress = peer.getLocalAddressString(); LOG.debug("Number of active connections is: {}", datanode.getXceiverCount()); }
@Override // ClientDatanodeProtocol public void refreshNamenodes() throws IOException { checkSuperuserPrivilege(); setConf(new Configuration()); refreshNamenodes(getConf()); }
IOStreamPair connectToDN(DatanodeInfo datanodeID, int timeout, ExtendedBlock block, Token<BlockTokenIdentifier> blockToken) throws IOException { return DFSUtilClient.connectToDN(datanodeID, timeout, getConf(), saslClient, NetUtils.getDefaultSocketFactory(getConf()), false, getDataEncryptionKeyFactoryForBlock(block), blockToken); }
/** * Stores the information related to a namenode in the cluster */ public static class NameNodeInfo { final NameNode nameNode; final Configuration conf; final String nameserviceId; final String nnId; StartupOption startOpt; NameNodeInfo(NameNode nn, String nameserviceId, String nnId, StartupOption startOpt, Configuration conf) { this.nameNode = nn; this.nameserviceId = nameserviceId; this.nnId = nnId; this.startOpt = startOpt; this.conf = conf; } public void setStartOpt(StartupOption startOpt) { this.startOpt = startOpt; } }
/** 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 { DataNode dn = instantiateDataNode(args, conf, resources); if (dn != null) { dn.runDatanodeDaemon(); } return dn; }
DatanodeRegistration bpReg = bpos.bpRegistration; InterDatanodeProtocol datanode = bpReg.equals(id)? this: DataNode.createInterDataNodeProtocolProxy(id, getConf(), dnConf.socketTimeout, dnConf.connectToDnViaHostname); ReplicaRecoveryInfo info = callInitReplicaRecovery(datanode, rBlock); if (info != null && info.getGenerationStamp() >= block.getGenerationStamp() && syncBlock(rBlock, syncList);
try { InterDatanodeProtocol datanode = dnRegistration.equals(id)? this: DataNode.createInterDataNodeProtocolProxy(id, getConf()); BlockMetaDataInfo info = datanode.getBlockMetaDataInfo(block); if (info != null && info.getGenerationStamp() >= block.getGenerationStamp()) { block.setNumBytes(minlength); return syncBlock(block, syncList, closeFile); } finally { synchronized (ongoingRecovery) {
/** * Check whether the datanode can be started. */ private boolean canStartDataNode(Configuration conf) throws IOException { DataNode dn = null; try { dn = DataNode.createDataNode(new String[]{}, conf); } catch(IOException e) { if (e instanceof java.net.BindException) return false; throw e; } finally { if(dn != null) dn.shutdown(); } return true; }
peer.setWriteTimeout(datanode.getDnConf().socketWriteTimeout); InputStream input = socketIn; try { IOStreamPair saslStreams = datanode.saslServer.receive(peer, socketOut, socketIn, datanode.getXferAddress().getPort(), datanode.getDatanodeId()); input = new BufferedInputStream(saslStreams.in, smallBufferSize); (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0)); } catch (Throwable t) { String s = datanode.getDisplayName() + ":DataXceiver error processing " + ((op == null) ? "unknown" : op.name()) + " operation " + " src: " + remoteAddress + " dst: " + localAddress; collectThreadLocalStates(); LOG.debug("{}:Number of active connections is: {}", datanode.getDisplayName(), datanode.getXceiverCount()); updateCurrentThreadName("Cleaning up"); if (peer != null) {
try { InterDatanodeProtocol datanode; if (getDNRegistrationForNS(namespaceId).equals(id)) { LOG.info("Skipping IDNPP creation for local id " + id + " when recovering " + block); } else { LOG.info("Creating IDNPP for non-local id " + id + " (dnReg=" + getDNRegistrationForNS(namespaceId) + ") when recovering " + block); datanode = DataNode.createInterDataNodeProtocolProxy( id, getConf(), socketTimeout); datanodeProxies.add(datanode); throwIfAfterTime(deadline); BlockRecoveryInfo info = datanode.startBlockRecovery(namespaceId, block); if (info == null) { stopAllProxies(datanodeProxies); throw new IOException("All datanodes failed: block=" + block + ", datanodeids=" + Arrays.asList(datanodeids)); block.setNumBytes(minlength); return syncBlock(namespaceId, block, syncList, closeFile, datanodeProxies, deadline); } finally {
private void sendLifeline() throws IOException { StorageReport[] reports = dn.getFSDataset().getStorageReports(bpos.getBlockPoolId()); if (LOG.isDebugEnabled()) { LOG.debug("Sending lifeline with " + reports.length + " storage " + " reports from service actor: " + BPServiceActor.this); } VolumeFailureSummary volumeFailureSummary = dn.getFSDataset() .getVolumeFailureSummary(); int numFailedVolumes = volumeFailureSummary != null ? volumeFailureSummary.getFailedStorageLocations().length : 0; lifelineNamenode.sendLifeline(bpRegistration, reports, dn.getFSDataset().getCacheCapacity(), dn.getFSDataset().getCacheUsed(), dn.getXmitsInProgress(), dn.getXceiverCount(), numFailedVolumes, volumeFailureSummary); } }
CommonConfigurationKeys.HADOOP_SKIP_VERSION_CHECK_DEFAULT); InetSocketAddress socAddr = DataNode.getStreamingAddr(conf); int tmpPort = socAddr.getPort(); storage = new DataStorage(); conf); NamespaceInfo nsInfo = handshake(); StartupOption startOpt = getStartupOption(conf); assert startOpt != null : "Startup option must be set."; conf.getBoolean("dfs.datanode.simulateddatastorage", false); if (simulatedFSDataset) { setNewStorageID(dnRegistration); dnRegistration.storageInfo.layoutVersion = FSConstants.LAYOUT_VERSION; dnRegistration.storageInfo.namespaceID = nsInfo.namespaceID; this.registerMXBean(conf); // register the MXBean for DataNode InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf); String infoHost = infoSocAddr.getHostName(); int tmpInfoPort = infoSocAddr.getPort();
final SecureResources resources) throws IOException { super(conf); this.tracer = createTracer(conf); this.tracerConfigurationManager = new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf); hostName = getHostName(conf); LOG.info("Configured hostname is {}", hostName); startDataNode(dataDirs, resources); } catch (IOException ie) { shutdown(); throw ie; initOOBTimeout(); this.storageLocationChecker = storageLocationChecker;
/** * Allows submission of a disk balancer Job. * @param planID - Hash value of the plan. * @param planVersion - Plan version, reserved for future use. We have only * version 1 now. * @param planFile - Plan file name * @param planData - Actual plan data in json format * @throws IOException */ @Override public void submitDiskBalancerPlan(String planID, long planVersion, String planFile, String planData, boolean skipDateCheck) throws IOException { checkSuperuserPrivilege(); if (getStartupOption(getConf()) != StartupOption.REGULAR) { throw new DiskBalancerException( "Datanode is in special state, e.g. Upgrade/Rollback etc." + " Disk balancing not permitted.", DiskBalancerException.Result.DATANODE_STATUS_NOT_REGULAR); } getDiskBalancer().submitPlan(planID, planVersion, planFile, planData, skipDateCheck); }
@Override public void onCompleteLazyPersist(String bpId, long blockId, long creationTime, File[] savedFiles, FsVolumeImpl targetVolume) { try (AutoCloseableLock lock = datasetLock.acquire()) { ramDiskReplicaTracker.recordEndLazyPersist(bpId, blockId, savedFiles); targetVolume.incDfsUsedAndNumBlocks(bpId, savedFiles[0].length() + savedFiles[1].length()); // Update metrics (ignore the metadata file size) datanode.getMetrics().incrRamDiskBlocksLazyPersisted(); datanode.getMetrics().incrRamDiskBytesLazyPersisted(savedFiles[1].length()); datanode.getMetrics().addRamDiskBlocksLazyPersistWindowMs( Time.monotonicNow() - creationTime); if (LOG.isDebugEnabled()) { LOG.debug("LazyWriter: Finish persisting RamDisk block: " + " block pool Id: " + bpId + " block id: " + blockId + " to block file " + savedFiles[1] + " and meta file " + savedFiles[0] + " on target volume " + targetVolume); } } }
public BlockScanner(DataNode datanode) { this(datanode, datanode.getConf()); }
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) { datanode.join(); } else { errorCode = 1; } } catch (Throwable e) { LOG.error("Exception in secureMain", e); terminate(1, e); } finally { // We need to terminate the process here because either shutdown was called // or some disk related conditions like volumes tolerated or volumes required // 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(errorCode); } }