diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index cd830faf6bd..51516878f15 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -30,6 +30,10 @@ Release 2.4.0 - UNRELEASED HDFS-5514. FSNamesystem's fsLock should allow custom implementation (daryn) + HDFS-2832. Heterogeneous Storages support in HDFS phase 1 - treat DataNode + as a collection of storages (see breakdown of tasks below for features and + contributors). + IMPROVEMENTS HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu) @@ -251,6 +255,131 @@ Release 2.4.0 - UNRELEASED HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to HTTP_ONLY. (Haohui Mai via jing9) +BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS + + HDFS-4985. Add storage type to the protocol and expose it in block report + and block locations. (Arpit Agarwal) + + HDFS-5115. Make StorageID a UUID. (Arpit Agarwal) + + HDFS-5000. DataNode configuration should allow specifying storage type. + (Arpit Agarwal) + + HDFS-4987. Namenode changes to track multiple storages per datanode. + (szetszwo) + + HDFS-5154. Fix TestBlockManager and TestDatanodeDescriptor after HDFS-4987. + (Junping Du via szetszwo) + + HDFS-5009. Include storage information in the LocatedBlock. (szetszwo) + + HDFS-5134. Move blockContentsStale, heartbeatedSinceFailover and + firstBlockReport from DatanodeDescriptor to DatanodeStorageInfo; and + fix a synchronization problem in DatanodeStorageInfo. (szetszwo) + + HDFS-5157. Add StorageType to FsVolume. (Junping Du via szetszwo) + + HDFS-4990. Change BlockPlacementPolicy to choose storages instead of + datanodes. (szetszwo) + + HDFS-5232. Protocol changes to transmit StorageUuid. (Arpit Agarwal) + + HDFS-5233. Use Datanode UUID to identify Datanodes. (Arpit Agarwal) + + HDFS-5222. Move block schedule information from DatanodeDescriptor to + DatanodeStorageInfo. (szetszwo) + + HDFS-4988. Datanode must support all the volumes as individual storages. + (Arpit Agarwal) + + HDFS-5377. Heartbeats from Datandode should include one storage report + per storage directory. (Arpit Agarwal) + + HDFS-5398. NameNode changes to process storage reports per storage + directory. (Arpit Agarwal) + + HDFS-5390. Send one incremental block report per storage directory. + (Arpit Agarwal) + + HDFS-5401. Fix NPE in Directory Scanner. (Arpit Agarwal) + + HDFS-5417. Fix storage IDs in PBHelper and UpgradeUtilities. (szetszwo) + + HDFS-5214. Fix NPEs in BlockManager and DirectoryScanner. (Arpit Agarwal) + + HDFS-5435. File append fails to initialize storageIDs. (Junping Du via + Arpit Agarwal) + + HDFS-5437. Fix TestBlockReport and TestBPOfferService failures. (Arpit + Agarwal) + + HDFS-5447. Fix TestJspHelper. (Arpit Agarwal) + + HDFS-5452. Fix TestReplicationPolicy and TestBlocksScheduledCounter. + + HDFS-5448. Datanode should generate its ID on first registration. (Arpit + Agarwal) + + HDFS-5448. Fix break caused by previous checkin for HDFS-5448. (Arpit + Agarwal) + + HDFS-5455. NN should update storageMap on first heartbeat. (Arpit Agarwal) + + HDFS-5457. Fix TestDatanodeRegistration, TestFsck and TestAddBlockRetry. + (Contributed by szetszwo) + + HDFS-5466. Update storage IDs when the pipeline is updated. (Contributed + by szetszwo) + + HDFS-5439. Fix TestPendingReplication. (Contributed by Junping Du, Arpit + Agarwal) + + HDFS-5470. Add back trunk's reportDiff algorithm to the branch. + (Contributed by szetszwo) + + HDFS-5472. Fix TestDatanodeManager, TestSafeMode and + TestNNThroughputBenchmark (Contributed by szetszwo) + + HDFS-5475. NN incorrectly tracks more than one replica per DN. (Arpit + Agarwal) + + HDFS-5481. Fix TestDataNodeVolumeFailure in branch HDFS-2832. (Contributed + by Junping Du) + + HDFS-5480. Update Balancer for HDFS-2832. (Contributed by szetszwo) + + HDFS-5486. Fix TestNameNodeMetrics for HDFS-2832. (Arpit Agarwal) + + HDFS-5491. Update editsStored for HDFS-2832. (Arpit Agarwal) + + HDFS-5494. Fix findbugs warnings for HDFS-2832. (Arpit Agarwal) + + HDFS-5508. Fix compilation error after merge. (Contributed by szetszwo) + + HDFS-5501. Fix pendingReceivedRequests tracking in BPServiceActor. (Arpit + Agarwal) + + HDFS-5510. Fix a findbug warning in DataStorage.java on HDFS-2832 branch. + (Junping Du via Arpit Agarwal) + + HDFS-5515. Fix TestDFSStartupVersions for HDFS-2832. (Arpit Agarwal) + + HDFS-5527. Fix TestUnderReplicatedBlocks on branch HDFS-2832. (Arpit + Agarwal) + + HDFS-5547. Fix build break after merge from trunk to HDFS-2832. (Arpit + Agarwal) + + HDFS-5542. Fix TODO and clean up the code in HDFS-2832. (Contributed by + szetszwo) + + HDFS-5559. Fix TestDatanodeConfig in HDFS-2832. (Contributed by szetszwo) + + HDFS-5484. StorageType and State in DatanodeStorageInfo in NameNode is + not accurate. (Eric Sirianni via Arpit Agarwal) + + HDFS-5648. Get rid of FsDatasetImpl#perVolumeReplicaMap. (Arpit Agarwal) + Release 2.3.0 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 3909b3e1c0f..0e02614eff6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -1383,7 +1383,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead, * deadNodes and added currentNode again. Thats ok. */ deadNodes.remove(oldNode); } - if (!oldNode.getStorageID().equals(newNode.getStorageID())) { + if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) { currentNode = newNode; return true; } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 60e6c274345..726866bfa99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -321,6 +321,7 @@ public class DFSOutputStream extends FSOutputSummer private DataInputStream blockReplyStream; private ResponseProcessor response = null; private volatile DatanodeInfo[] nodes = null; // list of targets for current block + private volatile String[] storageIDs = null; private LoadingCache excludedNodes = CacheBuilder.newBuilder() .expireAfterWrite( @@ -411,7 +412,7 @@ public class DFSOutputStream extends FSOutputSummer } // setup pipeline to append to the last block XXX retries?? - nodes = lastBlock.getLocations(); + setPipeline(lastBlock); errorIndex = -1; // no errors yet. if (nodes.length < 1) { throw new IOException("Unable to retrieve blocks locations " + @@ -420,6 +421,14 @@ public class DFSOutputStream extends FSOutputSummer } } + + private void setPipeline(LocatedBlock lb) { + setPipeline(lb.getLocations(), lb.getStorageIDs()); + } + private void setPipeline(DatanodeInfo[] nodes, String[] storageIDs) { + this.nodes = nodes; + this.storageIDs = storageIDs; + } private void setFavoredNodes(String[] favoredNodes) { this.favoredNodes = favoredNodes; @@ -443,7 +452,7 @@ public class DFSOutputStream extends FSOutputSummer this.setName("DataStreamer for file " + src); closeResponder(); closeStream(); - nodes = null; + setPipeline(null, null); stage = BlockConstructionStage.PIPELINE_SETUP_CREATE; } @@ -510,7 +519,7 @@ public class DFSOutputStream extends FSOutputSummer if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Allocating new block"); } - nodes = nextBlockOutputStream(); + setPipeline(nextBlockOutputStream()); initDataStreaming(); } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) { if(DFSClient.LOG.isDebugEnabled()) { @@ -922,9 +931,10 @@ public class DFSOutputStream extends FSOutputSummer //get a new datanode final DatanodeInfo[] original = nodes; final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode( - src, block, nodes, failed.toArray(new DatanodeInfo[failed.size()]), + src, block, nodes, storageIDs, + failed.toArray(new DatanodeInfo[failed.size()]), 1, dfsClient.clientName); - nodes = lb.getLocations(); + setPipeline(lb); //find the new datanode final int d = findNewDatanode(original); @@ -1024,7 +1034,14 @@ public class DFSOutputStream extends FSOutputSummer System.arraycopy(nodes, 0, newnodes, 0, errorIndex); System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex, newnodes.length-errorIndex); - nodes = newnodes; + + final String[] newStorageIDs = new String[newnodes.length]; + System.arraycopy(storageIDs, 0, newStorageIDs, 0, errorIndex); + System.arraycopy(storageIDs, errorIndex+1, newStorageIDs, errorIndex, + newStorageIDs.length-errorIndex); + + setPipeline(newnodes, newStorageIDs); + hasError = false; lastException.set(null); errorIndex = -1; @@ -1060,7 +1077,8 @@ public class DFSOutputStream extends FSOutputSummer // update pipeline at the namenode ExtendedBlock newBlock = new ExtendedBlock( block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS); - dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, nodes); + dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, + nodes, storageIDs); // update client side generation stamp block = newBlock; } @@ -1073,7 +1091,7 @@ public class DFSOutputStream extends FSOutputSummer * Must get block ID and the IDs of the destinations from the namenode. * Returns the list of target datanodes. */ - private DatanodeInfo[] nextBlockOutputStream() throws IOException { + private LocatedBlock nextBlockOutputStream() throws IOException { LocatedBlock lb = null; DatanodeInfo[] nodes = null; int count = dfsClient.getConf().nBlockWriteRetry; @@ -1115,7 +1133,7 @@ public class DFSOutputStream extends FSOutputSummer if (!success) { throw new IOException("Unable to create new block."); } - return nodes; + return lb; } // connects to the first datanode in the pipeline diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index 55af3d4f93e..3c8a628d706 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -145,6 +145,23 @@ public class DFSUtil { return SECURE_RANDOM.get(); } + /** Shuffle the elements in the given array. */ + public static T[] shuffle(final T[] array) { + if (array != null && array.length > 0) { + final Random random = getRandom(); + for (int n = array.length; n > 1; ) { + final int randomIndex = random.nextInt(n); + n--; + if (n != randomIndex) { + final T tmp = array[randomIndex]; + array[randomIndex] = array[n]; + array[n] = tmp; + } + } + } + return array; + } + /** * Compartor for sorting DataNodeInfo[] based on decommissioned states. * Decommissioned nodes are moved to the end of the array on sorting with diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java new file mode 100644 index 00000000000..3f250f86fa5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java @@ -0,0 +1,35 @@ +/** + * 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; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Defines the types of supported storage media. The default storage + * medium is assumed to be DISK. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public enum StorageType { + DISK, + SSD; + + public static StorageType DEFAULT = DISK; +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java index a6c2df46b50..a7f5ff16f75 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java @@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.protocol; import java.util.Iterator; import java.util.List; +import java.util.Random; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; @@ -250,33 +252,28 @@ public class BlockListAsLongs implements Iterable { } /** - * The block-id of the indexTh block - * @param index - the block whose block-id is desired - * @return the block-id + * Corrupt the generation stamp of the block with the given index. + * Not meant to be used outside of tests. */ - @Deprecated - public long getBlockId(final int index) { - return blockId(index); - } - - /** - * The block-len of the indexTh block - * @param index - the block whose block-len is desired - * @return - the block-len - */ - @Deprecated - public long getBlockLen(final int index) { - return blockLength(index); + @VisibleForTesting + public long corruptBlockGSForTesting(final int blockIndex, Random rand) { + long oldGS = blockList[index2BlockId(blockIndex) + 2]; + while (blockList[index2BlockId(blockIndex) + 2] == oldGS) { + blockList[index2BlockId(blockIndex) + 2] = rand.nextInt(); + } + return oldGS; } /** - * The generation stamp of the indexTh block - * @param index - the block whose block-len is desired - * @return - the generation stamp + * Corrupt the length of the block with the given index by truncation. + * Not meant to be used outside of tests. */ - @Deprecated - public long getBlockGenStamp(final int index) { - return blockGenerationStamp(index); + @VisibleForTesting + public long corruptBlockLengthForTesting(final int blockIndex, Random rand) { + long oldLength = blockList[index2BlockId(blockIndex) + 1]; + blockList[index2BlockId(blockIndex) + 1] = + rand.nextInt((int) oldLength - 1); + return oldLength; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java index 5789c3615eb..b2b45f3dbf2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java @@ -353,7 +353,8 @@ public interface ClientProtocol { */ @Idempotent public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk, - final DatanodeInfo[] existings, final DatanodeInfo[] excludes, + final DatanodeInfo[] existings, final String[] existingStorageIDs, + final DatanodeInfo[] excludes, final int numAdditionalNodes, final String clientName ) throws AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, IOException; @@ -982,7 +983,7 @@ public interface ClientProtocol { */ @AtMostOnce public void updatePipeline(String clientName, ExtendedBlock oldBlock, - ExtendedBlock newBlock, DatanodeID[] newNodes) + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs) throws IOException; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java index 9a012107b6d..c781e5b9ece 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hdfs.protocol; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import com.google.common.annotations.VisibleForTesting; + /** * This class represents the primary identifier for a Datanode. * Datanodes are identified by how they can be contacted (hostname @@ -40,37 +42,46 @@ public class DatanodeID implements Comparable { private String ipAddr; // IP address private String hostName; // hostname claimed by datanode private String peerHostName; // hostname from the actual connection - private String storageID; // unique per cluster storageID private int xferPort; // data streaming port private int infoPort; // info server port private int infoSecurePort; // info server port private int ipcPort; // IPC server port + /** + * UUID identifying a given datanode. For upgraded Datanodes this is the + * same as the StorageID that was previously used by this Datanode. + * For newly formatted Datanodes it is a UUID. + */ + private String datanodeUuid = null; + public DatanodeID(DatanodeID from) { this(from.getIpAddr(), from.getHostName(), - from.getStorageID(), + from.getDatanodeUuid(), from.getXferPort(), from.getInfoPort(), from.getInfoSecurePort(), from.getIpcPort()); this.peerHostName = from.getPeerHostName(); } - + /** * Create a DatanodeID * @param ipAddr IP * @param hostName hostname - * @param storageID data storage ID + * @param datanodeUuid data node ID, UUID for new Datanodes, may be the + * storage ID for pre-UUID datanodes. NULL if unknown + * e.g. if this is a new datanode. A new UUID will + * be assigned by the namenode. * @param xferPort data transfer port * @param infoPort info server port * @param ipcPort ipc server port */ - public DatanodeID(String ipAddr, String hostName, String storageID, + public DatanodeID(String ipAddr, String hostName, String datanodeUuid, int xferPort, int infoPort, int infoSecurePort, int ipcPort) { this.ipAddr = ipAddr; this.hostName = hostName; - this.storageID = storageID; + this.datanodeUuid = checkDatanodeUuid(datanodeUuid); this.xferPort = xferPort; this.infoPort = infoPort; this.infoSecurePort = infoSecurePort; @@ -85,8 +96,24 @@ public class DatanodeID implements Comparable { this.peerHostName = peerHostName; } - public void setStorageID(String storageID) { - this.storageID = storageID; + /** + * @return data node ID. + */ + public String getDatanodeUuid() { + return datanodeUuid; + } + + @VisibleForTesting + public void setDatanodeUuidForTesting(String datanodeUuid) { + this.datanodeUuid = datanodeUuid; + } + + private String checkDatanodeUuid(String uuid) { + if (uuid == null || uuid.isEmpty()) { + return null; + } else { + return uuid; + } } /** @@ -168,13 +195,6 @@ public class DatanodeID implements Comparable { return useHostname ? getIpcAddrWithHostname() : getIpcAddr(); } - /** - * @return data storage ID. - */ - public String getStorageID() { - return storageID; - } - /** * @return xferPort (the port for data streaming) */ @@ -212,12 +232,12 @@ public class DatanodeID implements Comparable { return false; } return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) && - storageID.equals(((DatanodeID)to).getStorageID())); + datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid())); } @Override public int hashCode() { - return getXferAddr().hashCode()^ storageID.hashCode(); + return getXferAddr().hashCode()^ datanodeUuid.hashCode(); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java index 7b5e30b9bad..d924c45269f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java @@ -108,7 +108,7 @@ public class DatanodeInfo extends DatanodeID implements Node { final long capacity, final long dfsUsed, final long remaining, final long blockPoolUsed, final long lastUpdate, final int xceiverCount, final AdminStates adminState) { - this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(), + this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(), nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(), nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed, lastUpdate, xceiverCount, location, adminState); @@ -116,12 +116,12 @@ public class DatanodeInfo extends DatanodeID implements Node { /** Constructor */ public DatanodeInfo(final String ipAddr, final String hostName, - final String storageID, final int xferPort, final int infoPort, + final String datanodeUuid, final int xferPort, final int infoPort, final int infoSecurePort, final int ipcPort, final long capacity, final long dfsUsed, final long remaining, final long blockPoolUsed, final long lastUpdate, final int xceiverCount, final String networkLocation, final AdminStates adminState) { - super(ipAddr, hostName, storageID, xferPort, infoPort, + super(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort, ipcPort); this.capacity = capacity; this.dfsUsed = dfsUsed; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java index d4c62c4c710..0f7460c645b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java @@ -106,8 +106,10 @@ public class LayoutVersion { SEQUENTIAL_BLOCK_ID(-46, "Allocate block IDs sequentially and store " + "block IDs in the edits log and image files"), EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " - + "enable rebuilding retry cache in case of HA failover"); - + + "enable rebuilding retry cache in case of HA failover"), + ADD_DATANODE_AND_STORAGE_UUIDS(-49, -47, "Replace StorageID with DatanodeUuid." + + " Use distinct StorageUuid per storage directory.", false); + final int lv; final int ancestorLV; final String description; @@ -247,3 +249,4 @@ public class LayoutVersion { throw new AssertionError("All layout versions are reserved."); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java index d9da5b845b7..e053641afed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java @@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.protocol; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.security.token.Token; /** @@ -34,6 +36,10 @@ public class LocatedBlock { private ExtendedBlock b; private long offset; // offset of the first byte of the block in the file private DatanodeInfo[] locs; + /** Storage ID for each replica */ + private String[] storageIDs; + // Storage type for each replica, if reported. + private StorageType[] storageTypes; // corrupt flag is true if all of the replicas of a block are corrupt. // else false. If block has few corrupt replicas, they are filtered and // their locations are not part of this object @@ -44,11 +50,30 @@ public class LocatedBlock { this(b, locs, -1, false); // startOffset is unknown } - public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) { - this(b, locs, startOffset, false); + public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, + boolean corrupt) { + this(b, locs, null, null, startOffset, corrupt); } - public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, + public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages) { + this(b, storages, -1, false); // startOffset is unknown + } + + public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, + String[] storageIDs, StorageType[] storageTypes) { + this(b, locs, storageIDs, storageTypes, -1, false); + } + + public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages, + long startOffset, boolean corrupt) { + this(b, DatanodeStorageInfo.toDatanodeInfos(storages), + DatanodeStorageInfo.toStorageIDs(storages), + DatanodeStorageInfo.toStorageTypes(storages), + startOffset, corrupt); // startOffset is unknown + } + + public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs, + StorageType[] storageTypes, long startOffset, boolean corrupt) { this.b = b; this.offset = startOffset; @@ -58,6 +83,8 @@ public class LocatedBlock { } else { this.locs = locs; } + this.storageIDs = storageIDs; + this.storageTypes = storageTypes; } public Token getBlockToken() { @@ -75,7 +102,15 @@ public class LocatedBlock { public DatanodeInfo[] getLocations() { return locs; } + + public StorageType[] getStorageTypes() { + return storageTypes; + } + public String[] getStorageIDs() { + return storageIDs; + } + public long getStartOffset() { return offset; } @@ -106,3 +141,4 @@ public class LocatedBlock { + "}"; } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java index eabdd22a974..1ef7455539e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java @@ -51,7 +51,7 @@ public class UnregisteredNodeException extends IOException { */ public UnregisteredNodeException(DatanodeID nodeID, DatanodeInfo storedNode) { super("Data node " + nodeID + " is attempting to report storage ID " - + nodeID.getStorageID() + ". Node " + + nodeID.getDatanodeUuid() + ". Node " + storedNode + " is expected to serve this storage."); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java index d7a18a60ac8..8290a4b4e43 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java @@ -405,14 +405,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements throws ServiceException { try { List existingList = req.getExistingsList(); + List existingStorageIDsList = req.getExistingStorageUuidsList(); List excludesList = req.getExcludesList(); - LocatedBlock result = server.getAdditionalDatanode( - req.getSrc(), PBHelper.convert(req.getBlk()), + LocatedBlock result = server.getAdditionalDatanode(req.getSrc(), + PBHelper.convert(req.getBlk()), PBHelper.convert(existingList.toArray( new DatanodeInfoProto[existingList.size()])), + existingStorageIDsList.toArray( + new String[existingStorageIDsList.size()]), PBHelper.convert(excludesList.toArray( new DatanodeInfoProto[excludesList.size()])), - req.getNumAdditionalNodes(), req.getClientName()); + req.getNumAdditionalNodes(), req.getClientName()); return GetAdditionalDatanodeResponseProto.newBuilder().setBlock( PBHelper.convert(result)) .build(); @@ -813,10 +816,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements UpdatePipelineRequestProto req) throws ServiceException { try { List newNodes = req.getNewNodesList(); - server - .updatePipeline(req.getClientName(), PBHelper.convert(req - .getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper - .convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()]))); + List newStorageIDs = req.getStorageIDsList(); + server.updatePipeline(req.getClientName(), + PBHelper.convert(req.getOldBlock()), + PBHelper.convert(req.getNewBlock()), + PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])), + newStorageIDs.toArray(new String[newStorageIDs.size()])); return VOID_UPDATEPIPELINE_RESPONSE; } catch (IOException e) { throw new ServiceException(e); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java index b5be61a3bcd..a58485be09f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java @@ -335,7 +335,8 @@ public class ClientNamenodeProtocolTranslatorPB implements @Override public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk, - DatanodeInfo[] existings, DatanodeInfo[] excludes, + DatanodeInfo[] existings, String[] existingStorageIDs, + DatanodeInfo[] excludes, int numAdditionalNodes, String clientName) throws AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, IOException { @@ -344,6 +345,7 @@ public class ClientNamenodeProtocolTranslatorPB implements .setSrc(src) .setBlk(PBHelper.convert(blk)) .addAllExistings(PBHelper.convert(existings)) + .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs)) .addAllExcludes(PBHelper.convert(excludes)) .setNumAdditionalNodes(numAdditionalNodes) .setClientName(clientName) @@ -780,12 +782,13 @@ public class ClientNamenodeProtocolTranslatorPB implements @Override public void updatePipeline(String clientName, ExtendedBlock oldBlock, - ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException { + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException { UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder() .setClientName(clientName) .setOldBlock(PBHelper.convert(oldBlock)) .setNewBlock(PBHelper.convert(newBlock)) .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes))) + .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs)) .build(); try { rpcProxy.updatePipeline(null, req); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java index fd4cc4b01c5..315ad92d049 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java @@ -213,7 +213,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) { StorageReceivedDeletedBlocksProto.Builder repBuilder = StorageReceivedDeletedBlocksProto.newBuilder(); - repBuilder.setStorageID(storageBlock.getStorageID()); + repBuilder.setStorageUuid(storageBlock.getStorageID()); for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) { repBuilder.addBlocks(PBHelper.convert(rdBlock)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java index 3e424602fa9..339a03d59d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlo import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto; -import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto; @@ -100,14 +99,8 @@ public class DatanodeProtocolServerSideTranslatorPB implements HeartbeatRequestProto request) throws ServiceException { HeartbeatResponse response; try { - List list = request.getReportsList(); - StorageReport[] report = new StorageReport[list.size()]; - int i = 0; - for (StorageReportProto p : list) { - report[i++] = new StorageReport(p.getStorageID(), p.getFailed(), - p.getCapacity(), p.getDfsUsed(), p.getRemaining(), - p.getBlockPoolUsed()); - } + final StorageReport[] report = PBHelper.convertStorageReports( + request.getReportsList()); response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()), report, request.getXmitsInProgress(), request.getXceiverCount(), request.getFailedVolumes()); @@ -174,7 +167,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements for (int j = 0; j < list.size(); j++) { rdBlocks[j] = PBHelper.convert(list.get(j)); } - info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks); + info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageUuid(), rdBlocks); } try { impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java index 8f3eed96852..087c697c587 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java @@ -82,6 +82,6 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements throw new ServiceException(e); } return UpdateReplicaUnderRecoveryResponseProto.newBuilder() - .setStorageID(storageID).build(); + .setStorageUuid(storageID).build(); } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java index 4e518c35bfa..5174d861882 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java @@ -109,7 +109,7 @@ public class InterDatanodeProtocolTranslatorPB implements .setNewLength(newLength).setRecoveryId(recoveryId).build(); try { return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req - ).getStorageID(); + ).getStorageUuid(); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java index 2d1b441d08a..faf42837e31 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java @@ -25,12 +25,14 @@ import java.util.Arrays; import java.util.EnumSet; import java.util.List; +import com.google.common.base.Preconditions; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; @@ -42,12 +44,12 @@ 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.SafeModeAction; -import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry; -import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType; 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.SnapshotDiffReport.DiffReportEntry; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto; @@ -104,6 +106,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto; import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto; import org.apache.hadoop.hdfs.security.token.block.BlockKey; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; @@ -221,17 +225,20 @@ public class PBHelper { // DatanodeId public static DatanodeID convert(DatanodeIDProto dn) { - return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(), + return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(), dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn .getInfoSecurePort() : 0, dn.getIpcPort()); } public static DatanodeIDProto convert(DatanodeID dn) { + // For wire compatibility with older versions we transmit the StorageID + // which is the same as the DatanodeUuid. Since StorageID is a required + // field we pass the empty string if the DatanodeUuid is not yet known. return DatanodeIDProto.newBuilder() .setIpAddr(dn.getIpAddr()) .setHostName(dn.getHostName()) - .setStorageID(dn.getStorageID()) .setXferPort(dn.getXferPort()) + .setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "") .setInfoPort(dn.getInfoPort()) .setInfoSecurePort(dn.getInfoSecurePort()) .setIpcPort(dn.getIpcPort()).build(); @@ -273,12 +280,16 @@ public class PBHelper { public static BlockWithLocationsProto convert(BlockWithLocations blk) { return BlockWithLocationsProto.newBuilder() .setBlock(convert(blk.getBlock())) - .addAllStorageIDs(Arrays.asList(blk.getStorageIDs())).build(); + .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids())) + .addAllStorageUuids(Arrays.asList(blk.getStorageIDs())).build(); } public static BlockWithLocations convert(BlockWithLocationsProto b) { - return new BlockWithLocations(convert(b.getBlock()), b.getStorageIDsList() - .toArray(new String[0])); + final List datanodeUuids = b.getDatanodeUuidsList(); + final List storageUuids = b.getStorageUuidsList(); + return new BlockWithLocations(convert(b.getBlock()), + datanodeUuids.toArray(new String[datanodeUuids.size()]), + storageUuids.toArray(new String[storageUuids.size()])); } public static BlocksWithLocationsProto convert(BlocksWithLocations blks) { @@ -567,6 +578,18 @@ public class PBHelper { for (int i = 0; i < locs.length; i++) { builder.addLocs(i, PBHelper.convert(locs[i])); } + + StorageType[] storageTypes = b.getStorageTypes(); + if (storageTypes != null) { + for (int i = 0; i < storageTypes.length; ++i) { + builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i])); + } + } + final String[] storageIDs = b.getStorageIDs(); + if (storageIDs != null) { + builder.addAllStorageIDs(Arrays.asList(storageIDs)); + } + return builder.setB(PBHelper.convert(b.getBlock())) .setBlockToken(PBHelper.convert(b.getBlockToken())) .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build(); @@ -579,8 +602,27 @@ public class PBHelper { for (int i = 0; i < locs.size(); i++) { targets[i] = PBHelper.convert(locs.get(i)); } + + final int storageTypesCount = proto.getStorageTypesCount(); + final StorageType[] storageTypes; + if (storageTypesCount == 0) { + storageTypes = null; + } else { + Preconditions.checkState(storageTypesCount == locs.size()); + storageTypes = convertStorageTypeProtos(proto.getStorageTypesList()); + } + + final int storageIDsCount = proto.getStorageIDsCount(); + final String[] storageIDs; + if (storageIDsCount == 0) { + storageIDs = null; + } else { + Preconditions.checkState(storageIDsCount == locs.size()); + storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]); + } + LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, - proto.getOffset(), proto.getCorrupt()); + storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt()); lb.setBlockToken(PBHelper.convert(proto.getBlockToken())); return lb; } @@ -719,7 +761,8 @@ public class PBHelper { for (int i = 0; i < blocks.length; i++) { builder.addBlocks(PBHelper.convert(blocks[i])); } - builder.addAllTargets(PBHelper.convert(cmd.getTargets())); + builder.addAllTargets(convert(cmd.getTargets())) + .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs())); return builder.build(); } @@ -732,6 +775,15 @@ public class PBHelper { return Arrays.asList(ret); } + private static List convert(String[][] targetStorageUuids) { + StorageUuidsProto[] ret = new StorageUuidsProto[targetStorageUuids.length]; + for (int i = 0; i < targetStorageUuids.length; i++) { + ret[i] = StorageUuidsProto.newBuilder() + .addAllStorageUuids(Arrays.asList(targetStorageUuids[i])).build(); + } + return Arrays.asList(ret); + } + public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) { DatanodeCommandProto.Builder builder = DatanodeCommandProto.newBuilder(); if (datanodeCommand == null) { @@ -806,6 +858,14 @@ public class PBHelper { for (int i = 0; i < targetList.size(); i++) { targets[i] = PBHelper.convert(targetList.get(i)); } + + List targetStorageUuidsList = blkCmd.getTargetStorageUuidsList(); + String[][] targetStorageIDs = new String[targetStorageUuidsList.size()][]; + for(int i = 0; i < targetStorageIDs.length; i++) { + List storageIDs = targetStorageUuidsList.get(i).getStorageUuidsList(); + targetStorageIDs[i] = storageIDs.toArray(new String[storageIDs.size()]); + } + int action = DatanodeProtocol.DNA_UNKNOWN; switch (blkCmd.getAction()) { case TRANSFER: @@ -818,7 +878,8 @@ public class PBHelper { action = DatanodeProtocol.DNA_SHUTDOWN; break; } - return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets); + return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets, + targetStorageIDs); } public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) { @@ -1328,11 +1389,12 @@ public class PBHelper { public static DatanodeStorageProto convert(DatanodeStorage s) { return DatanodeStorageProto.newBuilder() - .setState(PBHelper.convert(s.getState())) - .setStorageID(s.getStorageID()).build(); + .setState(PBHelper.convertState(s.getState())) + .setStorageType(PBHelper.convertStorageType(s.getStorageType())) + .setStorageUuid(s.getStorageID()).build(); } - private static StorageState convert(State state) { + private static StorageState convertState(State state) { switch(state) { case READ_ONLY: return StorageState.READ_ONLY; @@ -1342,11 +1404,26 @@ public class PBHelper { } } - public static DatanodeStorage convert(DatanodeStorageProto s) { - return new DatanodeStorage(s.getStorageID(), PBHelper.convert(s.getState())); + private static StorageTypeProto convertStorageType( + StorageType type) { + switch(type) { + case DISK: + return StorageTypeProto.DISK; + case SSD: + return StorageTypeProto.SSD; + default: + throw new IllegalStateException( + "BUG: StorageType not found, type=" + type); + } } - private static State convert(StorageState state) { + public static DatanodeStorage convert(DatanodeStorageProto s) { + return new DatanodeStorage(s.getStorageUuid(), + PBHelper.convertState(s.getState()), + PBHelper.convertType(s.getStorageType())); + } + + private static State convertState(StorageState state) { switch(state) { case READ_ONLY: return DatanodeStorage.State.READ_ONLY; @@ -1356,11 +1433,47 @@ public class PBHelper { } } + private static StorageType convertType(StorageTypeProto type) { + switch(type) { + case DISK: + return StorageType.DISK; + case SSD: + return StorageType.SSD; + default: + throw new IllegalStateException( + "BUG: StorageTypeProto not found, type=" + type); + } + } + + private static StorageType[] convertStorageTypeProtos( + List storageTypesList) { + final StorageType[] storageTypes = new StorageType[storageTypesList.size()]; + for (int i = 0; i < storageTypes.length; ++i) { + storageTypes[i] = PBHelper.convertType(storageTypesList.get(i)); + } + return storageTypes; + } + public static StorageReportProto convert(StorageReport r) { return StorageReportProto.newBuilder() .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity()) .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining()) - .setStorageID(r.getStorageID()).build(); + .setStorageUuid(r.getStorageID()).build(); + } + + public static StorageReport convert(StorageReportProto p) { + return new StorageReport(p.getStorageUuid(), p.getFailed(), + p.getCapacity(), p.getDfsUsed(), p.getRemaining(), + p.getBlockPoolUsed()); + } + + public static StorageReport[] convertStorageReports( + List list) { + final StorageReport[] report = new StorageReport[list.size()]; + for (int i = 0; i < report.length; i++) { + report[i] = convert(list.get(i)); + } + return report; } public static JournalInfo convert(JournalInfoProto info) { @@ -1497,3 +1610,4 @@ public class PBHelper { return new ExactSizeInputStream(input, size); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java index 983e8048e99..95260d5c74d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdfs.server.balancer; import static com.google.common.base.Preconditions.checkArgument; - import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; import java.io.BufferedInputStream; @@ -221,9 +220,9 @@ public class Balancer { private Map globalBlockList = new HashMap(); private MovedBlocks movedBlocks = new MovedBlocks(); - // Map storage IDs to BalancerDatanodes - private Map datanodes - = new HashMap(); + /** Map (datanodeUuid -> BalancerDatanodes) */ + private final Map datanodeMap + = new HashMap(); private NetworkTopology cluster; @@ -241,6 +240,14 @@ public class Balancer { private PendingBlockMove() { } + @Override + public String toString() { + final Block b = block.getBlock(); + return b + " with size=" + b.getNumBytes() + " from " + + source.getDisplayName() + " to " + target.getDisplayName() + + " through " + proxySource.getDisplayName(); + } + /* choose a block & a proxy source for this pendingMove * whose source & target have already been chosen. * @@ -272,11 +279,7 @@ public class Balancer { if ( chooseProxySource() ) { movedBlocks.add(block); if (LOG.isDebugEnabled()) { - LOG.debug("Decided to move block "+ block.getBlockId() - +" with a length of "+StringUtils.byteDesc(block.getNumBytes()) - + " bytes from " + source.getDisplayName() - + " to " + target.getDisplayName() - + " using proxy source " + proxySource.getDisplayName() ); + LOG.debug("Decided to move " + this); } return true; } @@ -353,17 +356,9 @@ public class Balancer { sendRequest(out); receiveResponse(in); bytesMoved.inc(block.getNumBytes()); - LOG.info( "Moving block " + block.getBlock().getBlockId() + - " from "+ source.getDisplayName() + " to " + - target.getDisplayName() + " through " + - proxySource.getDisplayName() + - " is succeeded." ); + LOG.info("Successfully moved " + this); } catch (IOException e) { - LOG.warn("Error moving block "+block.getBlockId()+ - " from " + source.getDisplayName() + " to " + - target.getDisplayName() + " through " + - proxySource.getDisplayName() + - ": "+e.getMessage()); + LOG.warn("Failed to move " + this + ": " + e.getMessage()); } finally { IOUtils.closeStream(out); IOUtils.closeStream(in); @@ -415,9 +410,7 @@ public class Balancer { @Override public void run() { if (LOG.isDebugEnabled()) { - LOG.debug("Starting moving "+ block.getBlockId() + - " from " + proxySource.getDisplayName() + " to " + - target.getDisplayName()); + LOG.debug("Start moving " + PendingBlockMove.this); } dispatch(); } @@ -464,11 +457,6 @@ public class Balancer { return block; } - /* Return the block id */ - private long getBlockId() { - return block.getBlockId(); - } - /* Return the length of the block */ private long getNumBytes() { return block.getNumBytes(); @@ -552,7 +540,7 @@ public class Balancer { /* Get the storage id of the datanode */ protected String getStorageID() { - return datanode.getStorageID(); + return datanode.getDatanodeUuid(); } /** Decide if still need to move more bytes */ @@ -675,10 +663,10 @@ public class Balancer { synchronized (block) { // update locations - for ( String storageID : blk.getStorageIDs() ) { - BalancerDatanode datanode = datanodes.get(storageID); + for (String datanodeUuid : blk.getDatanodeUuids()) { + final BalancerDatanode d = datanodeMap.get(datanodeUuid); if (datanode != null) { // not an unknown datanode - block.addLocation(datanode); + block.addLocation(d); } } } @@ -852,16 +840,6 @@ public class Balancer { DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT)); } - /* Shuffle datanode array */ - static private void shuffleArray(DatanodeInfo[] datanodes) { - for (int i=datanodes.length; i>1; i--) { - int randomIndex = DFSUtil.getRandom().nextInt(i); - DatanodeInfo tmp = datanodes[randomIndex]; - datanodes[randomIndex] = datanodes[i-1]; - datanodes[i-1] = tmp; - } - } - /* Given a data node set, build a network topology and decide * over-utilized datanodes, above average utilized datanodes, * below average utilized datanodes, and underutilized datanodes. @@ -891,8 +869,7 @@ public class Balancer { * an increasing order or a decreasing order. */ long overLoadedBytes = 0L, underLoadedBytes = 0L; - shuffleArray(datanodes); - for (DatanodeInfo datanode : datanodes) { + for (DatanodeInfo datanode : DFSUtil.shuffle(datanodes)) { if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) { continue; // ignore decommissioning or decommissioned nodes } @@ -923,13 +900,13 @@ public class Balancer { datanodeS.utilization)*datanodeS.datanode.getCapacity()/100.0); } } - this.datanodes.put(datanode.getStorageID(), datanodeS); + datanodeMap.put(datanode.getDatanodeUuid(), datanodeS); } //logging logNodes(); - assert (this.datanodes.size() == + assert (this.datanodeMap.size() == overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+ aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size()) : "Mismatched number of datanodes"; @@ -1001,9 +978,9 @@ public class Balancer { // At last, match all remaining nodes chooseNodes(ANY_OTHER); - assert (datanodes.size() >= sources.size()+targets.size()) + assert (datanodeMap.size() >= sources.size()+targets.size()) : "Mismatched number of datanodes (" + - datanodes.size() + " total, " + + datanodeMap.size() + " total, " + sources.size() + " sources, " + targets.size() + " targets)"; @@ -1304,7 +1281,7 @@ public class Balancer { this.aboveAvgUtilizedDatanodes.clear(); this.belowAvgUtilizedDatanodes.clear(); this.underUtilizedDatanodes.clear(); - this.datanodes.clear(); + this.datanodeMap.clear(); this.sources.clear(); this.targets.clear(); this.policy.reset(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java index d82f5a29427..c0d4892dc77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java @@ -21,6 +21,7 @@ import java.util.LinkedList; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.util.LightWeightGSet; @@ -42,11 +43,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { private LightWeightGSet.LinkedElement nextLinkedElement; /** - * This array contains triplets of references. For each i-th datanode the - * block belongs to triplets[3*i] is the reference to the DatanodeDescriptor - * and triplets[3*i+1] and triplets[3*i+2] are references to the previous and - * the next blocks, respectively, in the list of blocks belonging to this - * data-node. + * This array contains triplets of references. For each i-th storage, the + * block belongs to triplets[3*i] is the reference to the + * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are + * references to the previous and the next blocks, respectively, in the list + * of blocks belonging to this storage. * * Using previous and next in Object triplets is done instead of a * {@link LinkedList} list to efficiently use memory. With LinkedList the cost @@ -88,10 +89,15 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { this.bc = bc; } - DatanodeDescriptor getDatanode(int index) { + public DatanodeDescriptor getDatanode(int index) { + DatanodeStorageInfo storage = getStorageInfo(index); + return storage == null ? null : storage.getDatanodeDescriptor(); + } + + DatanodeStorageInfo getStorageInfo(int index) { assert this.triplets != null : "BlockInfo is not initialized"; assert index >= 0 && index*3 < triplets.length : "Index is out of bound"; - return (DatanodeDescriptor)triplets[index*3]; + return (DatanodeStorageInfo)triplets[index*3]; } private BlockInfo getPrevious(int index) { @@ -114,14 +120,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { return info; } - private void setDatanode(int index, DatanodeDescriptor node, BlockInfo previous, - BlockInfo next) { + private void setStorageInfo(int index, DatanodeStorageInfo storage) { assert this.triplets != null : "BlockInfo is not initialized"; - int i = index * 3; - assert index >= 0 && i+2 < triplets.length : "Index is out of bound"; - triplets[i] = node; - triplets[i+1] = previous; - triplets[i+2] = next; + assert index >= 0 && index*3 < triplets.length : "Index is out of bound"; + triplets[index*3] = storage; } /** @@ -193,22 +195,34 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { } /** - * Add data-node this block belongs to. + * Add a {@link DatanodeStorageInfo} location for a block */ - public boolean addNode(DatanodeDescriptor node) { - if(findDatanode(node) >= 0) // the node is already there - return false; + boolean addStorage(DatanodeStorageInfo storage) { + boolean added = true; + int idx = findDatanode(storage.getDatanodeDescriptor()); + if(idx >= 0) { + if (getStorageInfo(idx) == storage) { // the storage is already there + return false; + } else { + // The block is on the DN but belongs to a different storage. + // Update our state. + removeStorage(storage); + added = false; // Just updating storage. Return false. + } + } // find the last null node int lastNode = ensureCapacity(1); - setDatanode(lastNode, node, null, null); - return true; + setStorageInfo(lastNode, storage); + setNext(lastNode, null); + setPrevious(lastNode, null); + return added; } /** - * Remove data-node from the block. + * Remove {@link DatanodeStorageInfo} location for a block */ - public boolean removeNode(DatanodeDescriptor node) { - int dnIndex = findDatanode(node); + boolean removeStorage(DatanodeStorageInfo storage) { + int dnIndex = findStorageInfo(storage); if(dnIndex < 0) // the node is not found return false; assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : @@ -216,10 +230,13 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { // find the last not null node int lastNode = numNodes()-1; // replace current node triplet by the lastNode one - setDatanode(dnIndex, getDatanode(lastNode), getPrevious(lastNode), - getNext(lastNode)); + setStorageInfo(dnIndex, getStorageInfo(lastNode)); + setNext(dnIndex, getNext(lastNode)); + setPrevious(dnIndex, getPrevious(lastNode)); // set the last triplet to null - setDatanode(lastNode, null, null, null); + setStorageInfo(lastNode, null); + setNext(lastNode, null); + setPrevious(lastNode, null); return true; } @@ -239,37 +256,70 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { } return -1; } + /** + * Find specified DatanodeStorageInfo. + * @param dn + * @return index or -1 if not found. + */ + int findStorageInfo(DatanodeInfo dn) { + int len = getCapacity(); + for(int idx = 0; idx < len; idx++) { + DatanodeStorageInfo cur = getStorageInfo(idx); + if(cur == null) + break; + if(cur.getDatanodeDescriptor() == dn) + return idx; + } + return -1; + } + + /** + * Find specified DatanodeStorageInfo. + * @param storageInfo + * @return index or -1 if not found. + */ + int findStorageInfo(DatanodeStorageInfo storageInfo) { + int len = getCapacity(); + for(int idx = 0; idx < len; idx++) { + DatanodeStorageInfo cur = getStorageInfo(idx); + if(cur == storageInfo) + return idx; + if(cur == null) + break; + } + return -1; + } /** * Insert this block into the head of the list of blocks - * related to the specified DatanodeDescriptor. + * related to the specified DatanodeStorageInfo. * If the head is null then form a new list. * @return current block as the new head of the list. */ - public BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) { - int dnIndex = this.findDatanode(dn); + BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) { + int dnIndex = this.findStorageInfo(storage); assert dnIndex >= 0 : "Data node is not found: current"; assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : "Block is already in the list and cannot be inserted."; this.setPrevious(dnIndex, null); this.setNext(dnIndex, head); if(head != null) - head.setPrevious(head.findDatanode(dn), this); + head.setPrevious(head.findStorageInfo(storage), this); return this; } /** * Remove this block from the list of blocks - * related to the specified DatanodeDescriptor. + * related to the specified DatanodeStorageInfo. * If this block is the head of the list then return the next block as * the new head. * @return the new head of the list or null if the list becomes - * empty after deletion. + * empy after deletion. */ - public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) { + BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) { if(head == null) return null; - int dnIndex = this.findDatanode(dn); + int dnIndex = this.findStorageInfo(storage); if(dnIndex < 0) // this block is not on the data-node list return head; @@ -278,9 +328,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { this.setNext(dnIndex, null); this.setPrevious(dnIndex, null); if(prev != null) - prev.setNext(prev.findDatanode(dn), next); + prev.setNext(prev.findStorageInfo(storage), next); if(next != null) - next.setPrevious(next.findDatanode(dn), prev); + next.setPrevious(next.findStorageInfo(storage), prev); if(this == head) // removing the head head = next; return head; @@ -292,7 +342,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { * * @return the new head of the list. */ - public BlockInfo moveBlockToHead(BlockInfo head, DatanodeDescriptor dn, + public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage, int curIndex, int headIndex) { if (head == this) { return this; @@ -301,9 +351,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { BlockInfo prev = this.setPrevious(curIndex, null); head.setPrevious(headIndex, this); - prev.setNext(prev.findDatanode(dn), next); + prev.setNext(prev.findStorageInfo(storage), next); if (next != null) - next.setPrevious(next.findDatanode(dn), prev); + next.setPrevious(next.findStorageInfo(storage), prev); return this; } @@ -331,10 +381,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement { * @return BlockInfoUnderConstruction - an under construction block. */ public BlockInfoUnderConstruction convertToBlockUnderConstruction( - BlockUCState s, DatanodeDescriptor[] targets) { + BlockUCState s, DatanodeStorageInfo[] targets) { if(isComplete()) { - return new BlockInfoUnderConstruction( - this, getBlockCollection().getBlockReplication(), s, targets); + return new BlockInfoUnderConstruction(this, + getBlockCollection().getBlockReplication(), s, targets); } // the block is already under construction BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java index 1b343747ecc..54e560ddeb8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java @@ -63,12 +63,12 @@ public class BlockInfoUnderConstruction extends BlockInfo { * corresponding replicas. */ static class ReplicaUnderConstruction extends Block { - private DatanodeDescriptor expectedLocation; + private final DatanodeStorageInfo expectedLocation; private ReplicaState state; private boolean chosenAsPrimary; ReplicaUnderConstruction(Block block, - DatanodeDescriptor target, + DatanodeStorageInfo target, ReplicaState state) { super(block); this.expectedLocation = target; @@ -82,7 +82,7 @@ public class BlockInfoUnderConstruction extends BlockInfo { * It is not guaranteed, but expected, that the data-node actually has * the replica. */ - DatanodeDescriptor getExpectedLocation() { + private DatanodeStorageInfo getExpectedStorageLocation() { return expectedLocation; } @@ -118,7 +118,7 @@ public class BlockInfoUnderConstruction extends BlockInfo { * Is data-node the replica belongs to alive. */ boolean isAlive() { - return expectedLocation.isAlive; + return expectedLocation.getDatanodeDescriptor().isAlive; } @Override // Block @@ -162,7 +162,7 @@ public class BlockInfoUnderConstruction extends BlockInfo { */ public BlockInfoUnderConstruction(Block blk, int replication, BlockUCState state, - DatanodeDescriptor[] targets) { + DatanodeStorageInfo[] targets) { super(blk, replication); assert getBlockUCState() != BlockUCState.COMPLETE : "BlockInfoUnderConstruction cannot be in COMPLETE state"; @@ -186,7 +186,7 @@ public class BlockInfoUnderConstruction extends BlockInfo { } /** Set expected locations */ - public void setExpectedLocations(DatanodeDescriptor[] targets) { + public void setExpectedLocations(DatanodeStorageInfo[] targets) { int numLocations = targets == null ? 0 : targets.length; this.replicas = new ArrayList(numLocations); for(int i = 0; i < numLocations; i++) @@ -198,12 +198,12 @@ public class BlockInfoUnderConstruction extends BlockInfo { * Create array of expected replica locations * (as has been assigned by chooseTargets()). */ - public DatanodeDescriptor[] getExpectedLocations() { + public DatanodeStorageInfo[] getExpectedStorageLocations() { int numLocations = replicas == null ? 0 : replicas.size(); - DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations]; + DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; for(int i = 0; i < numLocations; i++) - locations[i] = replicas.get(i).getExpectedLocation(); - return locations; + storages[i] = replicas.get(i).getExpectedStorageLocation(); + return storages; } /** Get the number of expected locations */ @@ -244,9 +244,9 @@ public class BlockInfoUnderConstruction extends BlockInfo { // The replica list is unchanged. for (ReplicaUnderConstruction r : replicas) { if (genStamp != r.getGenerationStamp()) { - r.getExpectedLocation().removeBlock(this); + r.getExpectedStorageLocation().removeBlock(this); NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica " - + "from location: " + r.getExpectedLocation()); + + "from location: " + r.getExpectedStorageLocation()); } } } @@ -302,31 +302,44 @@ public class BlockInfoUnderConstruction extends BlockInfo { if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) { continue; } - if (replicas.get(i).getExpectedLocation().getLastUpdate() > mostRecentLastUpdate) { - primary = replicas.get(i); + final ReplicaUnderConstruction ruc = replicas.get(i); + final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate(); + if (lastUpdate > mostRecentLastUpdate) { primaryNodeIndex = i; - mostRecentLastUpdate = primary.getExpectedLocation().getLastUpdate(); + primary = ruc; + mostRecentLastUpdate = lastUpdate; } } if (primary != null) { - primary.getExpectedLocation().addBlockToBeRecovered(this); + primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this); primary.setChosenAsPrimary(true); NameNode.blockStateChangeLog.info("BLOCK* " + this + " recovery started, primary=" + primary); } } - void addReplicaIfNotPresent(DatanodeDescriptor dn, + void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block, ReplicaState rState) { - for (ReplicaUnderConstruction r : replicas) { - if (r.getExpectedLocation() == dn) { + Iterator it = replicas.iterator(); + while (it.hasNext()) { + ReplicaUnderConstruction r = it.next(); + if(r.getExpectedStorageLocation() == storage) { // Record the gen stamp from the report r.setGenerationStamp(block.getGenerationStamp()); return; + } else if (r.getExpectedStorageLocation().getDatanodeDescriptor() == + storage.getDatanodeDescriptor()) { + + // The Datanode reported that the block is on a different storage + // than the one chosen by BlockPlacementPolicy. This can occur as + // we allow Datanodes to choose the target storage. Update our + // state by removing the stale entry and adding a new one. + it.remove(); + break; } } - replicas.add(new ReplicaUnderConstruction(block, dn, rState)); + replicas.add(new ReplicaUnderConstruction(block, storage, rState)); } @Override // BlockInfo diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index be6a393cea4..6a773c44c53 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Queue; import java.util.Set; import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; @@ -44,6 +45,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; +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.BlockListAsLongs.BlockReportIterator; @@ -70,8 +72,10 @@ import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; import org.apache.hadoop.net.Node; import org.apache.hadoop.security.UserGroupInformation; @@ -470,8 +474,8 @@ public class BlockManager { private void dumpBlockMeta(Block block, PrintWriter out) { List containingNodes = new ArrayList(); - List containingLiveReplicasNodes = - new ArrayList(); + List containingLiveReplicasNodes = + new ArrayList(); NumberReplicas numReplicas = new NumberReplicas(); // source node returned is not used @@ -498,9 +502,8 @@ public class BlockManager { Collection corruptNodes = corruptReplicas.getNodes(block); - for (Iterator jt = blocksMap.nodeIterator(block); - jt.hasNext();) { - DatanodeDescriptor node = jt.next(); + for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); String state = ""; if (corruptNodes != null && corruptNodes.contains(node)) { state = "(corrupt)"; @@ -509,7 +512,7 @@ public class BlockManager { state = "(decommissioned)"; } - if (node.areBlockContentsStale()) { + if (storage.areBlockContentsStale()) { state += " (block deletions maybe out of date)"; } out.print(" " + node + state + " : "); @@ -660,10 +663,9 @@ public class BlockManager { assert oldBlock == getStoredBlock(oldBlock) : "last block of the file is not in blocksMap"; - DatanodeDescriptor[] targets = getNodes(oldBlock); + DatanodeStorageInfo[] targets = getStorages(oldBlock); - BlockInfoUnderConstruction ucBlock = - bc.setLastBlock(oldBlock, targets); + BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets); blocksMap.replaceBlock(ucBlock); // Remove block from replication queue. @@ -673,9 +675,8 @@ public class BlockManager { pendingReplications.remove(ucBlock); // remove this block from the list of pending blocks to be deleted. - for (DatanodeDescriptor dd : targets) { - String datanodeId = dd.getStorageID(); - invalidateBlocks.remove(datanodeId, oldBlock); + for (DatanodeStorageInfo storage : targets) { + invalidateBlocks.remove(storage.getStorageID(), oldBlock); } // Adjust safe-mode totals, since under-construction blocks don't @@ -694,18 +695,17 @@ public class BlockManager { /** * Get all valid locations of the block */ - private List getValidLocations(Block block) { - ArrayList machineSet = - new ArrayList(blocksMap.numNodes(block)); - for(Iterator it = - blocksMap.nodeIterator(block); it.hasNext();) { - String storageID = it.next().getStorageID(); + private List getValidLocations(Block block) { + final List locations + = new ArrayList(blocksMap.numNodes(block)); + for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + final String storageID = storage.getStorageID(); // filter invalidate replicas if(!invalidateBlocks.contains(storageID, block)) { - machineSet.add(storageID); + locations.add(storage); } } - return machineSet; + return locations; } private List createLocatedBlockList(final BlockInfo[] blocks, @@ -773,9 +773,9 @@ public class BlockManager { + ", blk=" + blk); } final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk; - final DatanodeDescriptor[] locations = uc.getExpectedLocations(); + final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return new LocatedBlock(eb, locations, pos, false); + return new LocatedBlock(eb, storages, pos, false); } // get block locations @@ -790,15 +790,14 @@ public class BlockManager { final int numNodes = blocksMap.numNodes(blk); final boolean isCorrupt = numCorruptNodes == numNodes; final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes; - final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines]; + final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines]; int j = 0; if (numMachines > 0) { - for(Iterator it = blocksMap.nodeIterator(blk); - it.hasNext();) { - final DatanodeDescriptor d = it.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) { + final DatanodeDescriptor d = storage.getDatanodeDescriptor(); final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d); if (isCorrupt || (!isCorrupt && !replicaCorrupt)) - machines[j++] = d; + machines[j++] = storage; } } assert j == machines.length : @@ -990,13 +989,20 @@ public class BlockManager { } node.resetBlocks(); - invalidateBlocks.remove(node.getStorageID()); + invalidateBlocks.remove(node.getDatanodeUuid()); // If the DN hasn't block-reported since the most recent // failover, then we may have been holding up on processing // over-replicated blocks because of it. But we can now // process those blocks. - if (node.areBlockContentsStale()) { + boolean stale = false; + for(DatanodeStorageInfo storage : node.getStorageInfos()) { + if (storage.areBlockContentsStale()) { + stale = true; + break; + } + } + if (stale) { rescanPostponedMisreplicatedBlocks(); } } @@ -1015,9 +1021,8 @@ public class BlockManager { */ private void addToInvalidates(Block b) { StringBuilder datanodes = new StringBuilder(); - for (Iterator it = blocksMap.nodeIterator(b); it - .hasNext();) { - DatanodeDescriptor node = it.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); invalidateBlocks.add(b, node, false); datanodes.append(node).append(" "); } @@ -1035,7 +1040,7 @@ public class BlockManager { * for logging purposes */ public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk, - final DatanodeInfo dn, String reason) throws IOException { + final DatanodeInfo dn, String storageID, String reason) throws IOException { assert namesystem.hasWriteLock(); final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock()); if (storedBlock == null) { @@ -1048,11 +1053,11 @@ public class BlockManager { return; } markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason, - Reason.CORRUPTION_REPORTED), dn); + Reason.CORRUPTION_REPORTED), dn, storageID); } private void markBlockAsCorrupt(BlockToMarkCorrupt b, - DatanodeInfo dn) throws IOException { + DatanodeInfo dn, String storageID) throws IOException { DatanodeDescriptor node = getDatanodeManager().getDatanode(dn); if (node == null) { throw new IOException("Cannot mark " + b @@ -1068,7 +1073,7 @@ public class BlockManager { } // Add replica to the data-node if it is not already there - node.addBlock(b.stored); + node.addBlock(storageID, b.stored); // Add this replica to corruptReplicas Map corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason, @@ -1193,7 +1198,7 @@ public class BlockManager { @VisibleForTesting int computeReplicationWorkForBlocks(List> blocksToReplicate) { int requiredReplication, numEffectiveReplicas; - List containingNodes, liveReplicaNodes; + List containingNodes; DatanodeDescriptor srcNode; BlockCollection bc = null; int additionalReplRequired; @@ -1219,7 +1224,7 @@ public class BlockManager { // get a source data-node containingNodes = new ArrayList(); - liveReplicaNodes = new ArrayList(); + List liveReplicaNodes = new ArrayList(); NumberReplicas numReplicas = new NumberReplicas(); srcNode = chooseSourceDatanode( block, containingNodes, liveReplicaNodes, numReplicas, @@ -1279,7 +1284,7 @@ public class BlockManager { namesystem.writeLock(); try { for(ReplicationWork rw : work){ - DatanodeDescriptor[] targets = rw.targets; + final DatanodeStorageInfo[] targets = rw.targets; if(targets == null || targets.length == 0){ rw.targets = null; continue; @@ -1319,7 +1324,8 @@ public class BlockManager { if ( (numReplicas.liveReplicas() >= requiredReplication) && (!blockHasEnoughRacks(block)) ) { - if (rw.srcNode.getNetworkLocation().equals(targets[0].getNetworkLocation())) { + if (rw.srcNode.getNetworkLocation().equals( + targets[0].getDatanodeDescriptor().getNetworkLocation())) { //No use continuing, unless a new rack in this case continue; } @@ -1328,15 +1334,13 @@ public class BlockManager { // Add block to the to be replicated list rw.srcNode.addBlockToBeReplicated(block, targets); scheduledWork++; - - for (DatanodeDescriptor dn : targets) { - dn.incBlocksScheduled(); - } + DatanodeStorageInfo.incrementBlocksScheduled(targets); // Move the block-replication into a "pending" state. // The reason we use 'pending' is so we can retry // replications that fail after an appropriate amount of time. - pendingReplications.increment(block, targets); + pendingReplications.increment(block, + DatanodeStorageInfo.toDatanodeDescriptors(targets)); if(blockLog.isDebugEnabled()) { blockLog.debug( "BLOCK* block " + block @@ -1357,12 +1361,12 @@ public class BlockManager { if (blockLog.isInfoEnabled()) { // log which blocks have been scheduled for replication for(ReplicationWork rw : work){ - DatanodeDescriptor[] targets = rw.targets; + DatanodeStorageInfo[] targets = rw.targets; if (targets != null && targets.length != 0) { StringBuilder targetList = new StringBuilder("datanode(s)"); for (int k = 0; k < targets.length; k++) { targetList.append(' '); - targetList.append(targets[k]); + targetList.append(targets[k].getDatanodeDescriptor()); } blockLog.info("BLOCK* ask " + rw.srcNode + " to replicate " + rw.block + " to " + targetList); @@ -1386,15 +1390,16 @@ public class BlockManager { * @see BlockPlacementPolicy#chooseTarget(String, int, Node, * List, boolean, Set, long) */ - public DatanodeDescriptor[] chooseTarget(final String src, + public DatanodeStorageInfo[] chooseTarget(final String src, final int numOfReplicas, final DatanodeDescriptor client, final Set excludedNodes, final long blocksize, List favoredNodes) throws IOException { List favoredDatanodeDescriptors = getDatanodeDescriptors(favoredNodes); - final DatanodeDescriptor targets[] = blockplacement.chooseTarget(src, + final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src, numOfReplicas, client, excludedNodes, blocksize, - favoredDatanodeDescriptors); + // TODO: get storage type from file + favoredDatanodeDescriptors, StorageType.DEFAULT); if (targets.length < minReplication) { throw new IOException("File " + src + " could only be replicated to " + targets.length + " nodes instead of minReplication (=" @@ -1455,12 +1460,11 @@ public class BlockManager { * the given block */ @VisibleForTesting - DatanodeDescriptor chooseSourceDatanode( - Block block, - List containingNodes, - List nodesContainingLiveReplicas, - NumberReplicas numReplicas, - int priority) { + DatanodeDescriptor chooseSourceDatanode(Block block, + List containingNodes, + List nodesContainingLiveReplicas, + NumberReplicas numReplicas, + int priority) { containingNodes.clear(); nodesContainingLiveReplicas.clear(); DatanodeDescriptor srcNode = null; @@ -1468,12 +1472,12 @@ public class BlockManager { int decommissioned = 0; int corrupt = 0; int excess = 0; - Iterator it = blocksMap.nodeIterator(block); + Collection nodesCorrupt = corruptReplicas.getNodes(block); - while(it.hasNext()) { - DatanodeDescriptor node = it.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); LightWeightLinkedSet excessBlocks = - excessReplicateMap.get(node.getStorageID()); + excessReplicateMap.get(node.getDatanodeUuid()); if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) corrupt++; else if (node.isDecommissionInProgress() || node.isDecommissioned()) @@ -1481,7 +1485,7 @@ public class BlockManager { else if (excessBlocks != null && excessBlocks.contains(block)) { excess++; } else { - nodesContainingLiveReplicas.add(node); + nodesContainingLiveReplicas.add(storage); live++; } containingNodes.add(node); @@ -1613,10 +1617,11 @@ public class BlockManager { } /** - * The given datanode is reporting all its blocks. - * Update the (machine-->blocklist) and (block-->machinelist) maps. + * The given storage is reporting all its blocks. + * Update the (storage-->block list) and (block-->storage list) maps. */ - public void processReport(final DatanodeID nodeID, final String poolId, + public void processReport(final DatanodeID nodeID, + final DatanodeStorage storage, final String poolId, final BlockListAsLongs newReport) throws IOException { namesystem.writeLock(); final long startTime = Time.now(); //after acquiring write lock @@ -1630,26 +1635,28 @@ public class BlockManager { // To minimize startup time, we discard any second (or later) block reports // that we receive while still in startup phase. - if (namesystem.isInStartupSafeMode() && !node.isFirstBlockReport()) { + final DatanodeStorageInfo storageInfo = node.updateStorage(storage); + if (namesystem.isInStartupSafeMode() + && storageInfo.getBlockReportCount() > 0) { blockLog.info("BLOCK* processReport: " + "discarded non-initial block report from " + nodeID + " because namenode still in startup phase"); return; } - if (node.numBlocks() == 0) { + if (storageInfo.numBlocks() == 0) { // The first block report can be processed a lot more efficiently than // ordinary block reports. This shortens restart times. - processFirstBlockReport(node, newReport); + processFirstBlockReport(node, storage.getStorageID(), newReport); } else { - processReport(node, newReport); + processReport(node, storage, newReport); } // Now that we have an up-to-date block report, we know that any // deletions from a previous NN iteration have been accounted for. - boolean staleBefore = node.areBlockContentsStale(); - node.receivedBlockReport(); - if (staleBefore && !node.areBlockContentsStale()) { + boolean staleBefore = storageInfo.areBlockContentsStale(); + storageInfo.receivedBlockReport(); + if (staleBefore && !storageInfo.areBlockContentsStale()) { LOG.info("BLOCK* processReport: Received first block report from " + node + " after starting up or becoming active. Its block " + "contents are no longer considered stale"); @@ -1703,28 +1710,30 @@ public class BlockManager { } private void processReport(final DatanodeDescriptor node, + final DatanodeStorage storage, final BlockListAsLongs report) throws IOException { // Normal case: // Modify the (block-->datanode) map, according to the difference // between the old and new block report. // Collection toAdd = new LinkedList(); - Collection toRemove = new LinkedList(); + Collection toRemove = new TreeSet(); Collection toInvalidate = new LinkedList(); Collection toCorrupt = new LinkedList(); Collection toUC = new LinkedList(); - reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC); + reportDiff(node, storage, report, + toAdd, toRemove, toInvalidate, toCorrupt, toUC); // Process the blocks on each queue for (StatefulBlockInfo b : toUC) { - addStoredBlockUnderConstruction(b, node); + addStoredBlockUnderConstruction(b, node, storage.getStorageID()); } for (Block b : toRemove) { removeStoredBlock(b, node); } int numBlocksLogged = 0; for (BlockInfo b : toAdd) { - addStoredBlock(b, node, null, numBlocksLogged < maxNumBlocksToLog); + addStoredBlock(b, node, storage.getStorageID(), null, numBlocksLogged < maxNumBlocksToLog); numBlocksLogged++; } if (numBlocksLogged > maxNumBlocksToLog) { @@ -1738,7 +1747,7 @@ public class BlockManager { addToInvalidates(b, node); } for (BlockToMarkCorrupt b : toCorrupt) { - markBlockAsCorrupt(b, node); + markBlockAsCorrupt(b, node, storage.getStorageID()); } } @@ -1754,10 +1763,11 @@ public class BlockManager { * @throws IOException */ private void processFirstBlockReport(final DatanodeDescriptor node, + final String storageID, final BlockListAsLongs report) throws IOException { if (report == null) return; assert (namesystem.hasWriteLock()); - assert (node.numBlocks() == 0); + assert (node.getStorageInfo(storageID).numBlocks() == 0); BlockReportIterator itBR = report.getBlockReportIterator(); while(itBR.hasNext()) { @@ -1766,7 +1776,7 @@ public class BlockManager { if (shouldPostponeBlocksFromFuture && namesystem.isGenStampInFuture(iblk)) { - queueReportedBlock(node, iblk, reportedState, + queueReportedBlock(node, storageID, iblk, reportedState, QUEUE_REASON_FUTURE_GENSTAMP); continue; } @@ -1783,10 +1793,10 @@ public class BlockManager { if (shouldPostponeBlocksFromFuture) { // In the Standby, we may receive a block report for a file that we // just have an out-of-date gen-stamp or state for, for example. - queueReportedBlock(node, iblk, reportedState, + queueReportedBlock(node, storageID, iblk, reportedState, QUEUE_REASON_CORRUPT_STATE); } else { - markBlockAsCorrupt(c, node); + markBlockAsCorrupt(c, node, storageID); } continue; } @@ -1794,7 +1804,7 @@ public class BlockManager { // If block is under construction, add this replica to its list if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) { ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent( - node, iblk, reportedState); + node.getStorageInfo(storageID), iblk, reportedState); // OpenFileBlocks only inside snapshots also will be added to safemode // threshold. So we need to update such blocks to safemode // refer HDFS-5283 @@ -1807,22 +1817,25 @@ public class BlockManager { } //add replica if appropriate if (reportedState == ReplicaState.FINALIZED) { - addStoredBlockImmediate(storedBlock, node); + addStoredBlockImmediate(storedBlock, node, storageID); } } } - private void reportDiff(DatanodeDescriptor dn, + private void reportDiff(DatanodeDescriptor dn, DatanodeStorage storage, BlockListAsLongs newReport, Collection toAdd, // add to DatanodeDescriptor Collection toRemove, // remove from DatanodeDescriptor Collection toInvalidate, // should be removed from DN Collection toCorrupt, // add to corrupt replicas list Collection toUC) { // add to under-construction list + + final DatanodeStorageInfo storageInfo = dn.updateStorage(storage); + // place a delimiter in the list which separates blocks // that have been reported from those that have not BlockInfo delimiter = new BlockInfo(new Block(), 1); - boolean added = dn.addBlock(delimiter); + boolean added = storageInfo.addBlock(delimiter); assert added : "Delimiting block cannot be present in the node"; int headIndex = 0; //currently the delimiter is in the head of the list int curIndex; @@ -1834,20 +1847,21 @@ public class BlockManager { while(itBR.hasNext()) { Block iblk = itBR.next(); ReplicaState iState = itBR.getCurrentReplicaState(); - BlockInfo storedBlock = processReportedBlock(dn, iblk, iState, - toAdd, toInvalidate, toCorrupt, toUC); + BlockInfo storedBlock = processReportedBlock(dn, storage.getStorageID(), + iblk, iState, toAdd, toInvalidate, toCorrupt, toUC); + // move block to the head of the list if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) { - headIndex = dn.moveBlockToHead(storedBlock, curIndex, headIndex); + headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex); } } + // collect blocks that have not been reported // all of them are next to the delimiter - Iterator it = new DatanodeDescriptor.BlockIterator( - delimiter.getNext(0), dn); + Iterator it = storageInfo.new BlockIterator(delimiter.getNext(0)); while(it.hasNext()) toRemove.add(it.next()); - dn.removeBlock(delimiter); + storageInfo.removeBlock(delimiter); } /** @@ -1881,7 +1895,8 @@ public class BlockManager { * @return the up-to-date stored block, if it should be kept. * Otherwise, null. */ - private BlockInfo processReportedBlock(final DatanodeDescriptor dn, + private BlockInfo processReportedBlock(final DatanodeDescriptor dn, + final String storageID, final Block block, final ReplicaState reportedState, final Collection toAdd, final Collection toInvalidate, @@ -1896,7 +1911,7 @@ public class BlockManager { if (shouldPostponeBlocksFromFuture && namesystem.isGenStampInFuture(block)) { - queueReportedBlock(dn, block, reportedState, + queueReportedBlock(dn, storageID, block, reportedState, QUEUE_REASON_FUTURE_GENSTAMP); return null; } @@ -1917,7 +1932,7 @@ public class BlockManager { } // Ignore replicas already scheduled to be removed from the DN - if(invalidateBlocks.contains(dn.getStorageID(), block)) { + if(invalidateBlocks.contains(dn.getDatanodeUuid(), block)) { /* TODO: following assertion is incorrect, see HDFS-2668 assert storedBlock.findDatanode(dn) < 0 : "Block " + block + " in recentInvalidatesSet should not appear in DN " + dn; */ @@ -1931,7 +1946,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block // If the block is an out-of-date generation stamp or state, // but we're the standby, we shouldn't treat it as corrupt, // but instead just queue it for later processing. - queueReportedBlock(dn, storedBlock, reportedState, + queueReportedBlock(dn, storageID, storedBlock, reportedState, QUEUE_REASON_CORRUPT_STATE); } else { toCorrupt.add(c); @@ -1960,7 +1975,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block * standby node. @see PendingDataNodeMessages. * @param reason a textual reason to report in the debug logs */ - private void queueReportedBlock(DatanodeDescriptor dn, Block block, + private void queueReportedBlock(DatanodeDescriptor dn, String storageID, Block block, ReplicaState reportedState, String reason) { assert shouldPostponeBlocksFromFuture; @@ -1970,7 +1985,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block " from datanode " + dn + " for later processing " + "because " + reason + "."); } - pendingDNMessages.enqueueReportedBlock(dn, block, reportedState); + pendingDNMessages.enqueueReportedBlock(dn, storageID, block, reportedState); } /** @@ -1993,8 +2008,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block if (LOG.isDebugEnabled()) { LOG.debug("Processing previouly queued message " + rbi); } - processAndHandleReportedBlock( - rbi.getNode(), rbi.getBlock(), rbi.getReportedState(), null); + processAndHandleReportedBlock(rbi.getNode(), rbi.getStorageID(), + rbi.getBlock(), rbi.getReportedState(), null); } } @@ -2111,19 +2126,21 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block return false; } } - + void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, - DatanodeDescriptor node) throws IOException { + DatanodeDescriptor node, String storageID) throws IOException { BlockInfoUnderConstruction block = ucBlock.storedBlock; - block.addReplicaIfNotPresent(node, ucBlock.reportedBlock, ucBlock.reportedState); + block.addReplicaIfNotPresent(node.getStorageInfo(storageID), + ucBlock.reportedBlock, ucBlock.reportedState); + if (ucBlock.reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) { - addStoredBlock(block, node, null, true); + addStoredBlock(block, node, storageID, null, true); } - } - + } + /** * Faster version of - * {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, DatanodeDescriptor, boolean)} + * {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, String, DatanodeDescriptor, boolean)} * , intended for use with initial block report at startup. If not in startup * safe mode, will call standard addStoredBlock(). Assumes this method is * called "immediately" so there is no need to refresh the storedBlock from @@ -2134,17 +2151,17 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block * @throws IOException */ private void addStoredBlockImmediate(BlockInfo storedBlock, - DatanodeDescriptor node) + DatanodeDescriptor node, String storageID) throws IOException { assert (storedBlock != null && namesystem.hasWriteLock()); if (!namesystem.isInStartupSafeMode() || namesystem.isPopulatingReplQueues()) { - addStoredBlock(storedBlock, node, null, false); + addStoredBlock(storedBlock, node, storageID, null, false); return; } // just add it - node.addBlock(storedBlock); + node.addBlock(storageID, storedBlock); // Now check for completion of blocks and safe block count int numCurrentReplica = countLiveNodes(storedBlock); @@ -2167,6 +2184,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block */ private Block addStoredBlock(final BlockInfo block, DatanodeDescriptor node, + String storageID, DatanodeDescriptor delNodeHint, boolean logEveryBlock) throws IOException { @@ -2192,7 +2210,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block assert bc != null : "Block must belong to a file"; // add block to the datanode - boolean added = node.addBlock(storedBlock); + boolean added = node.addBlock(storageID, storedBlock); int curReplicaDelta; if (added) { @@ -2452,19 +2470,19 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block Collection nonExcess = new ArrayList(); Collection corruptNodes = corruptReplicas .getNodes(block); - for (Iterator it = blocksMap.nodeIterator(block); - it.hasNext();) { - DatanodeDescriptor cur = it.next(); - if (cur.areBlockContentsStale()) { + for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); + if (storage.areBlockContentsStale()) { LOG.info("BLOCK* processOverReplicatedBlock: " + "Postponing processing of over-replicated " + - block + " since datanode " + cur + " does not yet have up-to-date " + + block + " since storage + " + storage + + "datanode " + cur + " does not yet have up-to-date " + "block information."); postponeBlock(block); return; } LightWeightLinkedSet excessBlocks = excessReplicateMap.get(cur - .getStorageID()); + .getDatanodeUuid()); if (excessBlocks == null || !excessBlocks.contains(block)) { if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { // exclude corrupt replicas @@ -2553,10 +2571,10 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block private void addToExcessReplicate(DatanodeInfo dn, Block block) { assert namesystem.hasWriteLock(); - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(dn.getStorageID()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid()); if (excessBlocks == null) { excessBlocks = new LightWeightLinkedSet(); - excessReplicateMap.put(dn.getStorageID(), excessBlocks); + excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks); } if (excessBlocks.add(block)) { excessBlocksCount.incrementAndGet(); @@ -2604,7 +2622,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block // in "excess" there. // LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node - .getStorageID()); + .getDatanodeUuid()); if (excessBlocks != null) { if (excessBlocks.remove(block)) { excessBlocksCount.decrementAndGet(); @@ -2613,7 +2631,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block + block + " is removed from excessBlocks"); } if (excessBlocks.size() == 0) { - excessReplicateMap.remove(node.getStorageID()); + excessReplicateMap.remove(node.getDatanodeUuid()); } } } @@ -2628,12 +2646,18 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block * return the length of the added block; 0 if the block is not added */ private long addBlock(Block block, List results) { - final List machineSet = getValidLocations(block); - if(machineSet.size() == 0) { + final List locations = getValidLocations(block); + if(locations.size() == 0) { return 0; } else { - results.add(new BlockWithLocations(block, - machineSet.toArray(new String[machineSet.size()]))); + final String[] datanodeUuids = new String[locations.size()]; + final String[] storageIDs = new String[datanodeUuids.length]; + for(int i = 0; i < locations.size(); i++) { + final DatanodeStorageInfo s = locations.get(i); + datanodeUuids[i] = s.getDatanodeDescriptor().getDatanodeUuid(); + storageIDs[i] = s.getStorageID(); + } + results.add(new BlockWithLocations(block, datanodeUuids, storageIDs)); return block.getNumBytes(); } } @@ -2642,12 +2666,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block * The given node is reporting that it received a certain block. */ @VisibleForTesting - void addBlock(DatanodeDescriptor node, Block block, String delHint) + void addBlock(DatanodeDescriptor node, String storageID, Block block, String delHint) throws IOException { - // decrement number of blocks scheduled to this datanode. + // Decrement number of blocks scheduled to this datanode. // for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with // RECEIVED_BLOCK), we currently also decrease the approximate number. - node.decBlocksScheduled(); + node.decrementBlocksScheduled(); // get the deletion hint node DatanodeDescriptor delHintNode = null; @@ -2663,11 +2687,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block // Modify the blocks->datanode map and node's map. // pendingReplications.decrement(block, node); - processAndHandleReportedBlock(node, block, ReplicaState.FINALIZED, + processAndHandleReportedBlock(node, storageID, block, ReplicaState.FINALIZED, delHintNode); } - private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block, + private void processAndHandleReportedBlock(DatanodeDescriptor node, + String storageID, Block block, ReplicaState reportedState, DatanodeDescriptor delHintNode) throws IOException { // blockReceived reports a finalized block @@ -2675,7 +2700,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block Collection toInvalidate = new LinkedList(); Collection toCorrupt = new LinkedList(); Collection toUC = new LinkedList(); - processReportedBlock(node, block, reportedState, + processReportedBlock(node, storageID, block, reportedState, toAdd, toInvalidate, toCorrupt, toUC); // the block is only in one of the to-do lists // if it is in none then data-node already has it @@ -2683,11 +2708,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block : "The block should be only in one of the lists."; for (StatefulBlockInfo b : toUC) { - addStoredBlockUnderConstruction(b, node); + addStoredBlockUnderConstruction(b, node, storageID); } long numBlocksLogged = 0; for (BlockInfo b : toAdd) { - addStoredBlock(b, node, delHintNode, numBlocksLogged < maxNumBlocksToLog); + addStoredBlock(b, node, storageID, delHintNode, numBlocksLogged < maxNumBlocksToLog); numBlocksLogged++; } if (numBlocksLogged > maxNumBlocksToLog) { @@ -2701,7 +2726,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block addToInvalidates(b, node); } for (BlockToMarkCorrupt b : toCorrupt) { - markBlockAsCorrupt(b, node); + markBlockAsCorrupt(b, node, storageID); } } @@ -2713,7 +2738,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block * This method must be called with FSNamesystem lock held. */ public void processIncrementalBlockReport(final DatanodeID nodeID, - final String poolId, final ReceivedDeletedBlockInfo blockInfos[]) + final String poolId, final StorageReceivedDeletedBlocks srdb) throws IOException { assert namesystem.hasWriteLock(); int received = 0; @@ -2729,19 +2754,19 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block "Got incremental block report from unregistered or dead node"); } - for (ReceivedDeletedBlockInfo rdbi : blockInfos) { + for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) { switch (rdbi.getStatus()) { case DELETED_BLOCK: removeStoredBlock(rdbi.getBlock(), node); deleted++; break; case RECEIVED_BLOCK: - addBlock(node, rdbi.getBlock(), rdbi.getDelHints()); + addBlock(node, srdb.getStorageID(), rdbi.getBlock(), rdbi.getDelHints()); received++; break; case RECEIVING_BLOCK: receiving++; - processAndHandleReportedBlock(node, rdbi.getBlock(), + processAndHandleReportedBlock(node, srdb.getStorageID(), rdbi.getBlock(), ReplicaState.RBW, null); break; default: @@ -2773,24 +2798,23 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block int corrupt = 0; int excess = 0; int stale = 0; - Iterator nodeIter = blocksMap.nodeIterator(b); Collection nodesCorrupt = corruptReplicas.getNodes(b); - while (nodeIter.hasNext()) { - DatanodeDescriptor node = nodeIter.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) { corrupt++; } else if (node.isDecommissionInProgress() || node.isDecommissioned()) { decommissioned++; } else { LightWeightLinkedSet blocksExcess = excessReplicateMap.get(node - .getStorageID()); + .getDatanodeUuid()); if (blocksExcess != null && blocksExcess.contains(b)) { excess++; } else { live++; } } - if (node.areBlockContentsStale()) { + if (storage.areBlockContentsStale()) { stale++; } } @@ -2813,10 +2837,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block } // else proceed with fast case int live = 0; - Iterator nodeIter = blocksMap.nodeIterator(b); Collection nodesCorrupt = corruptReplicas.getNodes(b); - while (nodeIter.hasNext()) { - DatanodeDescriptor node = nodeIter.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node))) live++; } @@ -2828,10 +2851,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block int curReplicas = num.liveReplicas(); int curExpectedReplicas = getReplication(block); BlockCollection bc = blocksMap.getBlockCollection(block); - Iterator nodeIter = blocksMap.nodeIterator(block); StringBuilder nodeList = new StringBuilder(); - while (nodeIter.hasNext()) { - DatanodeDescriptor node = nodeIter.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); nodeList.append(node); nodeList.append(" "); } @@ -2936,14 +2958,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block return blocksMap.size(); } - public DatanodeDescriptor[] getNodes(BlockInfo block) { - DatanodeDescriptor[] nodes = - new DatanodeDescriptor[block.numNodes()]; - Iterator it = blocksMap.nodeIterator(block); - for (int i = 0; it != null && it.hasNext(); i++) { - nodes[i] = it.next(); + public DatanodeStorageInfo[] getStorages(BlockInfo block) { + final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()]; + int i = 0; + for(DatanodeStorageInfo s : blocksMap.getStorages(block)) { + storages[i++] = s; } - return nodes; + return storages; } public int getTotalBlocks() { @@ -3056,9 +3077,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block corruptReplicas.getNodes(b); int numExpectedReplicas = getReplication(b); String rackName = null; - for (Iterator it = blocksMap.nodeIterator(b); - it.hasNext();) { - DatanodeDescriptor cur = it.next(); + for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { + final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { if ((corruptNodes == null ) || !corruptNodes.contains(cur)) { if (numExpectedReplicas == 1 || @@ -3102,8 +3122,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block } /** @return an iterator of the datanodes. */ - public Iterator datanodeIterator(final Block block) { - return blocksMap.nodeIterator(block); + public Iterable getStorages(final Block block) { + return blocksMap.getStorages(block); } public int numCorruptReplicas(Block block) { @@ -3247,24 +3267,24 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block private DatanodeDescriptor srcNode; private List containingNodes; - private List liveReplicaNodes; + private List liveReplicaStorages; private int additionalReplRequired; - private DatanodeDescriptor targets[]; + private DatanodeStorageInfo targets[]; private int priority; public ReplicationWork(Block block, BlockCollection bc, DatanodeDescriptor srcNode, List containingNodes, - List liveReplicaNodes, + List liveReplicaStorages, int additionalReplRequired, int priority) { this.block = block; this.bc = bc; this.srcNode = srcNode; this.containingNodes = containingNodes; - this.liveReplicaNodes = liveReplicaNodes; + this.liveReplicaStorages = liveReplicaStorages; this.additionalReplRequired = additionalReplRequired; this.priority = priority; this.targets = null; @@ -3273,8 +3293,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block private void chooseTargets(BlockPlacementPolicy blockplacement, Set excludedNodes) { targets = blockplacement.chooseTarget(bc.getName(), - additionalReplRequired, srcNode, liveReplicaNodes, false, - excludedNodes, block.getNumBytes()); + additionalReplRequired, srcNode, liveReplicaStorages, false, + excludedNodes, block.getNumBytes(), StorageType.DEFAULT); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java index 87f5b4a59e7..93712e99d4a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; 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.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -67,13 +68,14 @@ public abstract class BlockPlacementPolicy { * @return array of DatanodeDescriptor instances chosen as target * and sorted as a pipeline. */ - public abstract DatanodeDescriptor[] chooseTarget(String srcPath, + public abstract DatanodeStorageInfo[] chooseTarget(String srcPath, int numOfReplicas, Node writer, - List chosenNodes, + List chosen, boolean returnChosenNodes, Set excludedNodes, - long blocksize); + long blocksize, + StorageType storageType); /** * Same as {@link #chooseTarget(String, int, Node, List, boolean, @@ -82,16 +84,19 @@ public abstract class BlockPlacementPolicy { * is only a hint and due to cluster state, namenode may not be * able to place the blocks on these datanodes. */ - DatanodeDescriptor[] chooseTarget(String src, + DatanodeStorageInfo[] chooseTarget(String src, int numOfReplicas, Node writer, Set excludedNodes, - long blocksize, List favoredNodes) { + long blocksize, + List favoredNodes, + StorageType storageType) { // This class does not provide the functionality of placing // a block in favored datanodes. The implementations of this class // are expected to provide this functionality + return chooseTarget(src, numOfReplicas, writer, - new ArrayList(numOfReplicas), false, excludedNodes, - blocksize); + new ArrayList(numOfReplicas), false, + excludedNodes, blocksize, storageType); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java index bbe3f3a5ecf..fa1bfe645dc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java @@ -29,11 +29,14 @@ import java.util.TreeSet; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.namenode.FSClusterStats; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.Node; import org.apache.hadoop.net.NodeBase; @@ -103,99 +106,101 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { } @Override - public DatanodeDescriptor[] chooseTarget(String srcPath, + public DatanodeStorageInfo[] chooseTarget(String srcPath, int numOfReplicas, Node writer, - List chosenNodes, + List chosenNodes, boolean returnChosenNodes, Set excludedNodes, - long blocksize) { + long blocksize, + StorageType storageType) { return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes, - excludedNodes, blocksize); + excludedNodes, blocksize, storageType); } @Override - DatanodeDescriptor[] chooseTarget(String src, + DatanodeStorageInfo[] chooseTarget(String src, int numOfReplicas, Node writer, Set excludedNodes, long blocksize, - List favoredNodes) { + List favoredNodes, + StorageType storageType) { try { if (favoredNodes == null || favoredNodes.size() == 0) { // Favored nodes not specified, fall back to regular block placement. return chooseTarget(src, numOfReplicas, writer, - new ArrayList(numOfReplicas), false, - excludedNodes, blocksize); + new ArrayList(numOfReplicas), false, + excludedNodes, blocksize, storageType); } Set favoriteAndExcludedNodes = excludedNodes == null ? new HashSet() : new HashSet(excludedNodes); // Choose favored nodes - List results = new ArrayList(); + List results = new ArrayList(); boolean avoidStaleNodes = stats != null && stats.isAvoidingStaleDataNodesForWrite(); for (int i = 0; i < Math.min(favoredNodes.size(), numOfReplicas); i++) { DatanodeDescriptor favoredNode = favoredNodes.get(i); // Choose a single node which is local to favoredNode. // 'results' is updated within chooseLocalNode - DatanodeDescriptor target = chooseLocalNode(favoredNode, + final DatanodeStorageInfo target = chooseLocalStorage(favoredNode, favoriteAndExcludedNodes, blocksize, - getMaxNodesPerRack(results, - numOfReplicas)[1], results, avoidStaleNodes); + getMaxNodesPerRack(results.size(), numOfReplicas)[1], + results, avoidStaleNodes, storageType); if (target == null) { LOG.warn("Could not find a target for file " + src + " with favored node " + favoredNode); continue; } - favoriteAndExcludedNodes.add(target); + favoriteAndExcludedNodes.add(target.getDatanodeDescriptor()); } if (results.size() < numOfReplicas) { // Not enough favored nodes, choose other nodes. numOfReplicas -= results.size(); - DatanodeDescriptor[] remainingTargets = + DatanodeStorageInfo[] remainingTargets = chooseTarget(src, numOfReplicas, writer, results, - false, favoriteAndExcludedNodes, blocksize); + false, favoriteAndExcludedNodes, blocksize, storageType); for (int i = 0; i < remainingTargets.length; i++) { results.add(remainingTargets[i]); } } return getPipeline(writer, - results.toArray(new DatanodeDescriptor[results.size()])); + results.toArray(new DatanodeStorageInfo[results.size()])); } catch (NotEnoughReplicasException nr) { // Fall back to regular block placement disregarding favored nodes hint return chooseTarget(src, numOfReplicas, writer, - new ArrayList(numOfReplicas), false, - excludedNodes, blocksize); + new ArrayList(numOfReplicas), false, + excludedNodes, blocksize, storageType); } } /** This is the implementation. */ - private DatanodeDescriptor[] chooseTarget(int numOfReplicas, + private DatanodeStorageInfo[] chooseTarget(int numOfReplicas, Node writer, - List chosenNodes, + List chosenStorage, boolean returnChosenNodes, Set excludedNodes, - long blocksize) { + long blocksize, + StorageType storageType) { if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) { - return DatanodeDescriptor.EMPTY_ARRAY; + return DatanodeStorageInfo.EMPTY_ARRAY; } if (excludedNodes == null) { excludedNodes = new HashSet(); } - int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas); + int[] result = getMaxNodesPerRack(chosenStorage.size(), numOfReplicas); numOfReplicas = result[0]; int maxNodesPerRack = result[1]; - List results = - new ArrayList(chosenNodes); - for (DatanodeDescriptor node:chosenNodes) { + final List results = new ArrayList(chosenStorage); + for (DatanodeStorageInfo storage : chosenStorage) { // add localMachine and related nodes to excludedNodes - addToExcludedNodes(node, excludedNodes); + addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes); } if (!clusterMap.contains(writer)) { @@ -205,20 +210,19 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { boolean avoidStaleNodes = (stats != null && stats.isAvoidingStaleDataNodesForWrite()); Node localNode = chooseTarget(numOfReplicas, writer, - excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes); + excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); if (!returnChosenNodes) { - results.removeAll(chosenNodes); + results.removeAll(chosenStorage); } // sorting nodes to form a pipeline return getPipeline((writer==null)?localNode:writer, - results.toArray(new DatanodeDescriptor[results.size()])); + results.toArray(new DatanodeStorageInfo[results.size()])); } - private int[] getMaxNodesPerRack(List chosenNodes, - int numOfReplicas) { + private int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) { int clusterSize = clusterMap.getNumOfLeaves(); - int totalNumOfReplicas = chosenNodes.size()+numOfReplicas; + int totalNumOfReplicas = numOfChosen + numOfReplicas; if (totalNumOfReplicas > clusterSize) { numOfReplicas -= (totalNumOfReplicas-clusterSize); totalNumOfReplicas = clusterSize; @@ -243,8 +247,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, - final boolean avoidStaleNodes) { + List results, + final boolean avoidStaleNodes, + StorageType storageType) { if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) { return writer; } @@ -253,7 +258,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { int numOfResults = results.size(); boolean newBlock = (numOfResults==0); if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) { - writer = results.get(0); + writer = results.get(0).getDatanodeDescriptor(); } // Keep a copy of original excludedNodes @@ -261,42 +266,49 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { new HashSet(excludedNodes) : null; try { if (numOfResults == 0) { - writer = chooseLocalNode(writer, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + writer = chooseLocalStorage(writer, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType) + .getDatanodeDescriptor(); if (--numOfReplicas == 0) { return writer; } } + final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor(); if (numOfResults <= 1) { - chooseRemoteRack(1, results.get(0), excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack, + results, avoidStaleNodes, storageType); if (--numOfReplicas == 0) { return writer; } } if (numOfResults <= 2) { - if (clusterMap.isOnSameRack(results.get(0), results.get(1))) { - chooseRemoteRack(1, results.get(0), excludedNodes, - blocksize, maxNodesPerRack, - results, avoidStaleNodes); + final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor(); + if (clusterMap.isOnSameRack(dn0, dn1)) { + chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack, + results, avoidStaleNodes, storageType); } else if (newBlock){ - chooseLocalRack(results.get(1), excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack, + results, avoidStaleNodes, storageType); } else { chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack, - results, avoidStaleNodes); + results, avoidStaleNodes, storageType); } if (--numOfReplicas == 0) { return writer; } } chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + maxNodesPerRack, results, avoidStaleNodes, storageType); } catch (NotEnoughReplicasException e) { - LOG.warn("Not able to place enough replicas, still in need of " - + (totalReplicasExpected - results.size()) + " to reach " - + totalReplicasExpected + "\n" - + e.getMessage()); + final String message = "Failed to place enough replicas, still in need of " + + (totalReplicasExpected - results.size()) + " to reach " + + totalReplicasExpected + "."; + if (LOG.isTraceEnabled()) { + LOG.trace(message, e); + } else { + LOG.warn(message + " " + e.getMessage()); + } + if (avoidStaleNodes) { // Retry chooseTarget again, this time not avoiding stale nodes. @@ -304,14 +316,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { // not chosen because they were stale, decommissioned, etc. // We need to additionally exclude the nodes that were added to the // result list in the successful calls to choose*() above. - for (Node node : results) { - oldExcludedNodes.add(node); + for (DatanodeStorageInfo resultStorage : results) { + oldExcludedNodes.add(resultStorage.getDatanodeDescriptor()); } // Set numOfReplicas, since it can get out of sync with the result list // if the NotEnoughReplicasException was thrown in chooseRandom(). numOfReplicas = totalReplicasExpected - results.size(); return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize, - maxNodesPerRack, results, false); + maxNodesPerRack, results, false, storageType); } } return writer; @@ -321,32 +333,36 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { * Choose localMachine as the target. * if localMachine is not available, * choose a node on the same rack - * @return the chosen node + * @return the chosen storage */ - protected DatanodeDescriptor chooseLocalNode(Node localMachine, + protected DatanodeStorageInfo chooseLocalStorage(Node localMachine, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, - boolean avoidStaleNodes) + List results, + boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { // if no local machine, randomly choose one node if (localMachine == null) return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + maxNodesPerRack, results, avoidStaleNodes, storageType); if (preferLocalNode && localMachine instanceof DatanodeDescriptor) { DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine; // otherwise try local machine first if (excludedNodes.add(localMachine)) { // was not in the excluded list - if (addIfIsGoodTarget(localDatanode, excludedNodes, blocksize, - maxNodesPerRack, false, results, avoidStaleNodes) >= 0) { - return localDatanode; + for(DatanodeStorageInfo localStorage : DFSUtil.shuffle( + localDatanode.getStorageInfos())) { + if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize, + maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) { + return localStorage; + } } } } // try a node on local rack return chooseLocalRack(localMachine, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + maxNodesPerRack, results, avoidStaleNodes, storageType); } /** @@ -368,27 +384,29 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { * in the cluster. * @return the chosen node */ - protected DatanodeDescriptor chooseLocalRack(Node localMachine, + protected DatanodeStorageInfo chooseLocalRack(Node localMachine, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, - boolean avoidStaleNodes) + List results, + boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { // no local machine, so choose a random machine if (localMachine == null) { return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + maxNodesPerRack, results, avoidStaleNodes, storageType); } // choose one from the local rack try { return chooseRandom(localMachine.getNetworkLocation(), excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); } catch (NotEnoughReplicasException e1) { // find the second replica DatanodeDescriptor newLocal=null; - for(DatanodeDescriptor nextNode : results) { + for(DatanodeStorageInfo resultStorage : results) { + DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor(); if (nextNode != localMachine) { newLocal = nextNode; break; @@ -397,16 +415,16 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { if (newLocal != null) { try { return chooseRandom(newLocal.getNetworkLocation(), excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); } catch(NotEnoughReplicasException e2) { //otherwise randomly choose one from the network return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + maxNodesPerRack, results, avoidStaleNodes, storageType); } } else { //otherwise randomly choose one from the network return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + maxNodesPerRack, results, avoidStaleNodes, storageType); } } } @@ -423,48 +441,51 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { Set excludedNodes, long blocksize, int maxReplicasPerRack, - List results, - boolean avoidStaleNodes) + List results, + boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { int oldNumOfReplicas = results.size(); // randomly choose one node from remote racks try { chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(), excludedNodes, blocksize, maxReplicasPerRack, results, - avoidStaleNodes); + avoidStaleNodes, storageType); } catch (NotEnoughReplicasException e) { chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas), localMachine.getNetworkLocation(), excludedNodes, blocksize, - maxReplicasPerRack, results, avoidStaleNodes); + maxReplicasPerRack, results, avoidStaleNodes, storageType); } } /** * Randomly choose one target from the given scope. - * @return the chosen node, if there is any. + * @return the chosen storage, if there is any. */ - protected DatanodeDescriptor chooseRandom(String scope, + protected DatanodeStorageInfo chooseRandom(String scope, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, - boolean avoidStaleNodes) + List results, + boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack, - results, avoidStaleNodes); + results, avoidStaleNodes, storageType); } /** * Randomly choose numOfReplicas targets from the given scope. * @return the first chosen node, if there is any. */ - protected DatanodeDescriptor chooseRandom(int numOfReplicas, + protected DatanodeStorageInfo chooseRandom(int numOfReplicas, String scope, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, - boolean avoidStaleNodes) + List results, + boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes( @@ -476,24 +497,32 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { builder.append("["); } boolean badTarget = false; - DatanodeDescriptor firstChosen = null; + DatanodeStorageInfo firstChosen = null; while(numOfReplicas > 0 && numOfAvailableNodes > 0) { DatanodeDescriptor chosenNode = (DatanodeDescriptor)clusterMap.chooseRandom(scope); if (excludedNodes.add(chosenNode)) { //was not in the excluded list numOfAvailableNodes--; - int newExcludedNodes = addIfIsGoodTarget(chosenNode, excludedNodes, - blocksize, maxNodesPerRack, considerLoad, results, avoidStaleNodes); - if (newExcludedNodes >= 0) { - numOfReplicas--; - if (firstChosen == null) { - firstChosen = chosenNode; + final DatanodeStorageInfo[] storages = DFSUtil.shuffle( + chosenNode.getStorageInfos()); + int i; + for(i = 0; i < storages.length; i++) { + final int newExcludedNodes = addIfIsGoodTarget(storages[i], + excludedNodes, blocksize, maxNodesPerRack, considerLoad, results, + avoidStaleNodes, storageType); + if (newExcludedNodes >= 0) { + numOfReplicas--; + if (firstChosen == null) { + firstChosen = storages[i]; + } + numOfAvailableNodes -= newExcludedNodes; + break; } - numOfAvailableNodes -= newExcludedNodes; - } else { - badTarget = true; } + + // If no candidate storage was found on this DN then set badTarget. + badTarget = (i == storages.length); } } @@ -512,43 +541,46 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { } /** - * If the given node is a good target, add it to the result list and + * If the given storage is a good target, add it to the result list and * update the set of excluded nodes. * @return -1 if the given is not a good target; * otherwise, return the number of nodes added to excludedNodes set. */ - int addIfIsGoodTarget(DatanodeDescriptor node, + int addIfIsGoodTarget(DatanodeStorageInfo storage, Set excludedNodes, long blockSize, int maxNodesPerRack, boolean considerLoad, - List results, - boolean avoidStaleNodes) { - if (isGoodTarget(node, blockSize, maxNodesPerRack, considerLoad, - results, avoidStaleNodes)) { - results.add(node); + List results, + boolean avoidStaleNodes, + StorageType storageType) { + if (isGoodTarget(storage, blockSize, maxNodesPerRack, considerLoad, + results, avoidStaleNodes, storageType)) { + results.add(storage); // add node and related nodes to excludedNode - return addToExcludedNodes(node, excludedNodes); + return addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes); } else { return -1; } } - private static void logNodeIsNotChosen(DatanodeDescriptor node, String reason) { + private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) { if (LOG.isDebugEnabled()) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); // build the error message for later use. debugLoggingBuilder.get() .append(node).append(": ") - .append("Node ").append(NodeBase.getPath(node)) + .append("Storage ").append(storage) + .append("at node ").append(NodeBase.getPath(node)) .append(" is not chosen because ") .append(reason); } } /** - * Determine if a node is a good target. + * Determine if a storage is a good target. * - * @param node The target node + * @param storage The target storage * @param blockSize Size of block * @param maxTargetPerRack Maximum number of targets per rack. The value of * this parameter depends on the number of racks in @@ -561,32 +593,43 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { * does not have too much load, * and the rack does not have too many nodes. */ - private boolean isGoodTarget(DatanodeDescriptor node, + private boolean isGoodTarget(DatanodeStorageInfo storage, long blockSize, int maxTargetPerRack, boolean considerLoad, - List results, - boolean avoidStaleNodes) { - // check if the node is (being) decommissed + List results, + boolean avoidStaleNodes, + StorageType storageType) { + if (storage.getStorageType() != storageType) { + logNodeIsNotChosen(storage, + "storage types do not match, where the expected storage type is " + + storageType); + return false; + } + if (storage.getState() == State.READ_ONLY) { + logNodeIsNotChosen(storage, "storage is read-only"); + return false; + } + DatanodeDescriptor node = storage.getDatanodeDescriptor(); + // check if the node is (being) decommissioned if (node.isDecommissionInProgress() || node.isDecommissioned()) { - logNodeIsNotChosen(node, "the node is (being) decommissioned "); + logNodeIsNotChosen(storage, "the node is (being) decommissioned "); return false; } if (avoidStaleNodes) { if (node.isStale(this.staleInterval)) { - logNodeIsNotChosen(node, "the node is stale "); + logNodeIsNotChosen(storage, "the node is stale "); return false; } } - long remaining = node.getRemaining() - - (node.getBlocksScheduled() * blockSize); - // check the remaining capacity of the target machine - if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) { - logNodeIsNotChosen(node, "the node does not have enough space "); + final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE; + final long scheduledSize = blockSize * node.getBlocksScheduled(); + if (requiredSize > node.getRemaining() - scheduledSize) { + logNodeIsNotChosen(storage, "the node does not have enough space "); return false; } - + // check the communication traffic of the target machine if (considerLoad) { double avgLoad = 0; @@ -595,7 +638,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { avgLoad = (double)stats.getTotalLoad()/size; } if (node.getXceiverCount() > (2.0 * avgLoad)) { - logNodeIsNotChosen(node, "the node is too busy "); + logNodeIsNotChosen(storage, "the node is too busy "); return false; } } @@ -603,13 +646,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { // check if the target rack has chosen too many nodes String rackname = node.getNetworkLocation(); int counter=1; - for(Node result : results) { - if (rackname.equals(result.getNetworkLocation())) { + for(DatanodeStorageInfo resultStorage : results) { + if (rackname.equals( + resultStorage.getDatanodeDescriptor().getNetworkLocation())) { counter++; } } if (counter>maxTargetPerRack) { - logNodeIsNotChosen(node, "the rack has too many chosen nodes "); + logNodeIsNotChosen(storage, "the rack has too many chosen nodes "); return false; } return true; @@ -621,37 +665,40 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { * starts from the writer and traverses all nodes * This is basically a traveling salesman problem. */ - private DatanodeDescriptor[] getPipeline(Node writer, - DatanodeDescriptor[] nodes) { - if (nodes.length==0) return nodes; - + private DatanodeStorageInfo[] getPipeline(Node writer, + DatanodeStorageInfo[] storages) { + if (storages.length == 0) { + return storages; + } + synchronized(clusterMap) { int index=0; if (writer == null || !clusterMap.contains(writer)) { - writer = nodes[0]; + writer = storages[0].getDatanodeDescriptor(); } - for(;indexcurrentDistance) { shortestDistance = currentDistance; - shortestNode = currentNode; + shortestStorage = storages[i]; shortestIndex = i; } } //switch position index & shortestIndex if (index != shortestIndex) { - nodes[shortestIndex] = nodes[index]; - nodes[index] = shortestNode; + storages[shortestIndex] = storages[index]; + storages[index] = shortestStorage; } - writer = shortestNode; + writer = shortestStorage.getDatanodeDescriptor(); } } - return nodes; + return storages; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java index bf591f5bad7..b2747bf1d35 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java @@ -25,6 +25,8 @@ import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.namenode.FSClusterStats; import org.apache.hadoop.net.NetworkTopology; @@ -63,83 +65,87 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau * @return the chosen node */ @Override - protected DatanodeDescriptor chooseLocalNode(Node localMachine, + protected DatanodeStorageInfo chooseLocalStorage(Node localMachine, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, boolean avoidStaleNodes) - throws NotEnoughReplicasException { + List results, boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { // if no local machine, randomly choose one node if (localMachine == null) return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); + // otherwise try local machine first if (localMachine instanceof DatanodeDescriptor) { DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine; - // otherwise try local machine first if (excludedNodes.add(localMachine)) { // was not in the excluded list - if (addIfIsGoodTarget(localDataNode, excludedNodes, blocksize, - maxNodesPerRack, false, results, avoidStaleNodes) >= 0) { - return localDataNode; + for(DatanodeStorageInfo localStorage : DFSUtil.shuffle( + localDataNode.getStorageInfos())) { + if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize, + maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) { + return localStorage; + } } } } // try a node on local node group - DatanodeDescriptor chosenNode = chooseLocalNodeGroup( + DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup( (NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); - if (chosenNode != null) { - return chosenNode; + blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); + if (chosenStorage != null) { + return chosenStorage; } // try a node on local rack return chooseLocalRack(localMachine, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); + } + + /** @return the node of the second replica */ + private static DatanodeDescriptor secondNode(Node localMachine, + List results) { + // find the second replica + for(DatanodeStorageInfo nextStorage : results) { + DatanodeDescriptor nextNode = nextStorage.getDatanodeDescriptor(); + if (nextNode != localMachine) { + return nextNode; + } + } + return null; } - /** - * {@inheritDoc} - */ @Override - protected DatanodeDescriptor chooseLocalRack(Node localMachine, + protected DatanodeStorageInfo chooseLocalRack(Node localMachine, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, boolean avoidStaleNodes) - throws NotEnoughReplicasException { + List results, boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { // no local machine, so choose a random machine if (localMachine == null) { - return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType); } // choose one from the local rack, but off-nodegroup try { - return chooseRandom(NetworkTopology.getFirstHalf( - localMachine.getNetworkLocation()), - excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes); + final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation()); + return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack, + results, avoidStaleNodes, storageType); } catch (NotEnoughReplicasException e1) { // find the second replica - DatanodeDescriptor newLocal=null; - for(DatanodeDescriptor nextNode : results) { - if (nextNode != localMachine) { - newLocal = nextNode; - break; - } - } + final DatanodeDescriptor newLocal = secondNode(localMachine, results); if (newLocal != null) { try { - return chooseRandom(clusterMap.getRack(newLocal.getNetworkLocation()), - excludedNodes, blocksize, maxNodesPerRack, results, - avoidStaleNodes); + return chooseRandom( + clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes, + blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); } catch(NotEnoughReplicasException e2) { //otherwise randomly choose one from the network - return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, - avoidStaleNodes); + return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType); } } else { //otherwise randomly choose one from the network - return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, - avoidStaleNodes); + return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType); } } } @@ -150,8 +156,9 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau @Override protected void chooseRemoteRack(int numOfReplicas, DatanodeDescriptor localMachine, Set excludedNodes, - long blocksize, int maxReplicasPerRack, List results, - boolean avoidStaleNodes) throws NotEnoughReplicasException { + long blocksize, int maxReplicasPerRack, List results, + boolean avoidStaleNodes, StorageType storageType) + throws NotEnoughReplicasException { int oldNumOfReplicas = results.size(); final String rackLocation = NetworkTopology.getFirstHalf( @@ -159,12 +166,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau try { // randomly choose from remote racks chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize, - maxReplicasPerRack, results, avoidStaleNodes); + maxReplicasPerRack, results, avoidStaleNodes, storageType); } catch (NotEnoughReplicasException e) { // fall back to the local rack chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas), rackLocation, excludedNodes, blocksize, - maxReplicasPerRack, results, avoidStaleNodes); + maxReplicasPerRack, results, avoidStaleNodes, storageType); } } @@ -174,43 +181,40 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau * if still no such node is available, choose a random node in the cluster. * @return the chosen node */ - private DatanodeDescriptor chooseLocalNodeGroup( + private DatanodeStorageInfo chooseLocalNodeGroup( NetworkTopologyWithNodeGroup clusterMap, Node localMachine, Set excludedNodes, long blocksize, int maxNodesPerRack, - List results, boolean avoidStaleNodes) - throws NotEnoughReplicasException { + List results, boolean avoidStaleNodes, + StorageType storageType) throws NotEnoughReplicasException { // no local machine, so choose a random machine if (localMachine == null) { - return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType); } // choose one from the local node group try { - return chooseRandom(clusterMap.getNodeGroup(localMachine.getNetworkLocation()), - excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes); + return chooseRandom( + clusterMap.getNodeGroup(localMachine.getNetworkLocation()), + excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, + storageType); } catch (NotEnoughReplicasException e1) { - // find the second replica - DatanodeDescriptor newLocal=null; - for(DatanodeDescriptor nextNode : results) { - if (nextNode != localMachine) { - newLocal = nextNode; - break; - } - } + final DatanodeDescriptor newLocal = secondNode(localMachine, results); if (newLocal != null) { try { - return chooseRandom(clusterMap.getNodeGroup(newLocal.getNetworkLocation()), - excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes); + return chooseRandom( + clusterMap.getNodeGroup(newLocal.getNetworkLocation()), + excludedNodes, blocksize, maxNodesPerRack, results, + avoidStaleNodes, storageType); } catch(NotEnoughReplicasException e2) { //otherwise randomly choose one from the network - return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType); } } else { //otherwise randomly choose one from the network - return chooseRandom(NodeBase.ROOT, excludedNodes, - blocksize, maxNodesPerRack, results, avoidStaleNodes); + return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageType); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java index 99dd965ef98..643dc8c2aaa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java @@ -29,11 +29,11 @@ import org.apache.hadoop.util.LightWeightGSet; * the datanodes that store the block. */ class BlocksMap { - private static class NodeIterator implements Iterator { + private static class StorageIterator implements Iterator { private BlockInfo blockInfo; private int nextIdx = 0; - NodeIterator(BlockInfo blkInfo) { + StorageIterator(BlockInfo blkInfo) { this.blockInfo = blkInfo; } @@ -44,8 +44,8 @@ class BlocksMap { } @Override - public DatanodeDescriptor next() { - return blockInfo.getDatanode(nextIdx++); + public DatanodeStorageInfo next() { + return blockInfo.getStorageInfo(nextIdx++); } @Override @@ -115,18 +115,23 @@ class BlocksMap { /** * Searches for the block in the BlocksMap and - * returns Iterator that iterates through the nodes the block belongs to. + * returns {@link Iterable} of the storages the block belongs to. */ - Iterator nodeIterator(Block b) { - return nodeIterator(blocks.get(b)); + Iterable getStorages(Block b) { + return getStorages(blocks.get(b)); } /** * For a block that has already been retrieved from the BlocksMap - * returns Iterator that iterates through the nodes the block belongs to. + * returns {@link Iterable} of the storages the block belongs to. */ - Iterator nodeIterator(BlockInfo storedBlock) { - return new NodeIterator(storedBlock); + Iterable getStorages(final BlockInfo storedBlock) { + return new Iterable() { + @Override + public Iterator iterator() { + return new StorageIterator(storedBlock); + } + }; } /** counts number of containing nodes. Better than using iterator. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index 4a7766aace5..721bc934653 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -17,20 +17,17 @@ */ package org.apache.hadoop.hdfs.server.blockmanagement; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; -import java.util.Set; -import java.util.TreeSet; +import java.util.*; +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.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.util.LightWeightHashSet; import org.apache.hadoop.util.Time; @@ -42,6 +39,7 @@ import org.apache.hadoop.util.Time; @InterfaceAudience.Private @InterfaceStability.Evolving public class DatanodeDescriptor extends DatanodeInfo { + public static final Log LOG = LogFactory.getLog(DatanodeDescriptor.class); public static final DatanodeDescriptor[] EMPTY_ARRAY = {}; // Stores status of decommissioning. @@ -53,9 +51,9 @@ public class DatanodeDescriptor extends DatanodeInfo { @InterfaceStability.Evolving public static class BlockTargetPair { public final Block block; - public final DatanodeDescriptor[] targets; + public final DatanodeStorageInfo[] targets; - BlockTargetPair(Block block, DatanodeDescriptor[] targets) { + BlockTargetPair(Block block, DatanodeStorageInfo[] targets) { this.block = block; this.targets = targets; } @@ -98,30 +96,14 @@ public class DatanodeDescriptor extends DatanodeInfo { } } - private volatile BlockInfo blockList = null; - private int numBlocks = 0; + private final Map storageMap = + new HashMap(); + // isAlive == heartbeats.contains(this) // This is an optimization, because contains takes O(n) time on Arraylist public boolean isAlive = false; public boolean needKeyUpdate = false; - /** - * Set to false on any NN failover, and reset to true - * whenever a block report is received. - */ - private boolean heartbeatedSinceFailover = false; - - /** - * At startup or at any failover, the DNs in the cluster may - * have pending block deletions from a previous incarnation - * of the NameNode. Thus, we consider their block contents - * stale until we have received a block report. When a DN - * is considered stale, any replicas on it are transitively - * considered stale. If any block has at least one stale replica, - * then no invalidations will be processed for this block. - * See HDFS-1972. - */ - private boolean blockContentsStale = true; // A system administrator can tune the balancer bandwidth parameter // (dfs.balance.bandwidthPerSec) dynamically by calling @@ -140,7 +122,7 @@ public class DatanodeDescriptor extends DatanodeInfo { private LightWeightHashSet invalidateBlocks = new LightWeightHashSet(); /* Variables for maintaining number of blocks scheduled to be written to - * this datanode. This count is approximate and might be slightly bigger + * this storage. This count is approximate and might be slightly bigger * in case of errors (e.g. datanode does not report if an error occurs * while writing the block). */ @@ -150,9 +132,6 @@ public class DatanodeDescriptor extends DatanodeInfo { private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min private int volumeFailures = 0; - /** Set to false after processing first block report */ - private boolean firstBlockReport = true; - /** * When set to true, the node is not in include list and is not allowed * to communicate with the namenode @@ -164,7 +143,8 @@ public class DatanodeDescriptor extends DatanodeInfo { * @param nodeID id of the data node */ public DatanodeDescriptor(DatanodeID nodeID) { - this(nodeID, 0L, 0L, 0L, 0L, 0, 0); + super(nodeID); + updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0); } /** @@ -174,95 +154,60 @@ public class DatanodeDescriptor extends DatanodeInfo { */ public DatanodeDescriptor(DatanodeID nodeID, String networkLocation) { - this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0, 0); - } - - /** - * DatanodeDescriptor constructor - * @param nodeID id of the data node - * @param capacity capacity of the data node - * @param dfsUsed space used by the data node - * @param remaining remaining capacity of the data node - * @param bpused space used by the block pool corresponding to this namenode - * @param xceiverCount # of data transfers at the data node - */ - public DatanodeDescriptor(DatanodeID nodeID, - long capacity, - long dfsUsed, - long remaining, - long bpused, - int xceiverCount, - int failedVolumes) { - super(nodeID); - updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount, - failedVolumes); - } - - /** - * DatanodeDescriptor constructor - * @param nodeID id of the data node - * @param networkLocation location of the data node in network - * @param capacity capacity of the data node, including space used by non-dfs - * @param dfsUsed the used space by dfs datanode - * @param remaining remaining capacity of the data node - * @param bpused space used by the block pool corresponding to this namenode - * @param xceiverCount # of data transfers at the data node - */ - public DatanodeDescriptor(DatanodeID nodeID, - String networkLocation, - long capacity, - long dfsUsed, - long remaining, - long bpused, - int xceiverCount, - int failedVolumes) { super(nodeID, networkLocation); - updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount, - failedVolumes); + updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0); } /** - * Add datanode to the block. - * Add block to the head of the list of blocks belonging to the data-node. + * Add data-node to the block. Add block to the head of the list of blocks + * belonging to the data-node. */ - public boolean addBlock(BlockInfo b) { - if(!b.addNode(this)) - return false; - // add to the head of the data-node list - blockList = b.listInsert(blockList, this); - numBlocks++; - return true; + public boolean addBlock(String storageID, BlockInfo b) { + DatanodeStorageInfo s = getStorageInfo(storageID); + if (s != null) { + return s.addBlock(b); + } + return false; } - - /** - * Remove block from the list of blocks belonging to the data-node. - * Remove datanode from the block. - */ - public boolean removeBlock(BlockInfo b) { - blockList = b.listRemove(blockList, this); - if ( b.removeNode(this) ) { - numBlocks--; - return true; - } else { - return false; + + DatanodeStorageInfo getStorageInfo(String storageID) { + synchronized (storageMap) { + return storageMap.get(storageID); + } + } + DatanodeStorageInfo[] getStorageInfos() { + synchronized (storageMap) { + final Collection storages = storageMap.values(); + return storages.toArray(new DatanodeStorageInfo[storages.size()]); } } /** - * Move block to the head of the list of blocks belonging to the data-node. - * @return the index of the head of the blockList + * Remove block from the list of blocks belonging to the data-node. Remove + * data-node from the block. */ - int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) { - blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex); - return curIndex; + boolean removeBlock(BlockInfo b) { + int index = b.findStorageInfo(this); + // if block exists on this datanode + if (index >= 0) { + DatanodeStorageInfo s = b.getStorageInfo(index); + if (s != null) { + return s.removeBlock(b); + } + } + return false; } - + /** - * Used for testing only - * @return the head of the blockList + * Remove block from the list of blocks belonging to the data-node. Remove + * data-node from the block. */ - protected BlockInfo getHead(){ - return blockList; + boolean removeBlock(String storageID, BlockInfo b) { + DatanodeStorageInfo s = getStorageInfo(storageID); + if (s != null) { + return s.removeBlock(b); + } + return false; } /** @@ -273,9 +218,12 @@ public class DatanodeDescriptor extends DatanodeInfo { * @return the new block */ public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) { - boolean done = removeBlock(oldBlock); + int index = oldBlock.findStorageInfo(this); + DatanodeStorageInfo s = oldBlock.getStorageInfo(index); + boolean done = s.removeBlock(oldBlock); assert done : "Old block should belong to the data-node when replacing"; - done = addBlock(newBlock); + + done = s.addBlock(newBlock); assert done : "New block should not belong to the data-node when replacing"; return newBlock; } @@ -286,7 +234,6 @@ public class DatanodeDescriptor extends DatanodeInfo { setBlockPoolUsed(0); setDfsUsed(0); setXceiverCount(0); - this.blockList = null; this.invalidateBlocks.clear(); this.volumeFailures = 0; } @@ -300,63 +247,95 @@ public class DatanodeDescriptor extends DatanodeInfo { } public int numBlocks() { - return numBlocks; + int blocks = 0; + for (DatanodeStorageInfo entry : getStorageInfos()) { + blocks += entry.numBlocks(); + } + return blocks; } /** * Updates stats from datanode heartbeat. */ - public void updateHeartbeat(long capacity, long dfsUsed, long remaining, - long blockPoolUsed, int xceiverCount, int volFailures) { - setCapacity(capacity); - setRemaining(remaining); - setBlockPoolUsed(blockPoolUsed); - setDfsUsed(dfsUsed); + public void updateHeartbeat(StorageReport[] reports, + int xceiverCount, int volFailures) { + long totalCapacity = 0; + long totalRemaining = 0; + long totalBlockPoolUsed = 0; + long totalDfsUsed = 0; + setXceiverCount(xceiverCount); setLastUpdate(Time.now()); this.volumeFailures = volFailures; - this.heartbeatedSinceFailover = true; + for (StorageReport report : reports) { + DatanodeStorageInfo storage = storageMap.get(report.getStorageID()); + if (storage == null) { + // This is seen during cluster initialization when the heartbeat + // is received before the initial block reports from each storage. + storage = updateStorage(new DatanodeStorage(report.getStorageID())); + } + storage.receivedHeartbeat(report); + totalCapacity += report.getCapacity(); + totalRemaining += report.getRemaining(); + totalBlockPoolUsed += report.getBlockPoolUsed(); + totalDfsUsed += report.getDfsUsed(); + } rollBlocksScheduled(getLastUpdate()); + + // Update total metrics for the node. + setCapacity(totalCapacity); + setRemaining(totalRemaining); + setBlockPoolUsed(totalBlockPoolUsed); + setDfsUsed(totalDfsUsed); } - /** - * Iterates over the list of blocks belonging to the datanode. - */ - public static class BlockIterator implements Iterator { - private BlockInfo current; - private DatanodeDescriptor node; - - BlockIterator(BlockInfo head, DatanodeDescriptor dn) { - this.current = head; - this.node = dn; + private static class BlockIterator implements Iterator { + private int index = 0; + private final List> iterators; + + private BlockIterator(final DatanodeStorageInfo... storages) { + List> iterators = new ArrayList>(); + for (DatanodeStorageInfo e : storages) { + iterators.add(e.getBlockIterator()); + } + this.iterators = Collections.unmodifiableList(iterators); } @Override public boolean hasNext() { - return current != null; + update(); + return !iterators.isEmpty() && iterators.get(index).hasNext(); } @Override public BlockInfo next() { - BlockInfo res = current; - current = current.getNext(current.findDatanode(node)); - return res; + update(); + return iterators.get(index).next(); } - + @Override - public void remove() { - throw new UnsupportedOperationException("Sorry. can't remove."); + public void remove() { + throw new UnsupportedOperationException("Remove unsupported."); + } + + private void update() { + while(index < iterators.size() - 1 && !iterators.get(index).hasNext()) { + index++; + } } } - public Iterator getBlockIterator() { - return new BlockIterator(this.blockList, this); + Iterator getBlockIterator() { + return new BlockIterator(getStorageInfos()); + } + Iterator getBlockIterator(final String storageID) { + return new BlockIterator(getStorageInfo(storageID)); } /** * Store block replication work. */ - void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) { + void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { assert(block != null && targets != null && targets.length > 0); replicateBlocks.offer(new BlockTargetPair(block, targets)); } @@ -431,18 +410,14 @@ public class DatanodeDescriptor extends DatanodeInfo { public int getBlocksScheduled() { return currApproxBlocksScheduled + prevApproxBlocksScheduled; } - - /** - * Increments counter for number of blocks scheduled. - */ - public void incBlocksScheduled() { + + /** Increment the number of blocks scheduled. */ + void incrementBlocksScheduled() { currApproxBlocksScheduled++; } - /** - * Decrements counter for number of blocks scheduled. - */ - void decBlocksScheduled() { + /** Decrement the number of blocks scheduled. */ + void decrementBlocksScheduled() { if (prevApproxBlocksScheduled > 0) { prevApproxBlocksScheduled--; } else if (currApproxBlocksScheduled > 0) { @@ -451,12 +426,9 @@ public class DatanodeDescriptor extends DatanodeInfo { // its ok if both counters are zero. } - /** - * Adjusts curr and prev number of blocks scheduled every few minutes. - */ + /** Adjusts curr and prev number of blocks scheduled every few minutes. */ private void rollBlocksScheduled(long now) { - if ((now - lastBlocksScheduledRollTime) > - BLOCKS_SCHEDULED_ROLL_INTERVAL) { + if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) { prevApproxBlocksScheduled = currApproxBlocksScheduled; currApproxBlocksScheduled = 0; lastBlocksScheduledRollTime = now; @@ -552,7 +524,11 @@ public class DatanodeDescriptor extends DatanodeInfo { @Override public void updateRegInfo(DatanodeID nodeReg) { super.updateRegInfo(nodeReg); - firstBlockReport = true; // must re-process IBR after re-registration + + // must re-process IBR after re-registration + for(DatanodeStorageInfo storage : getStorageInfos()) { + storage.setBlockReportCount(0); + } } /** @@ -569,26 +545,6 @@ public class DatanodeDescriptor extends DatanodeInfo { this.bandwidth = bandwidth; } - public boolean areBlockContentsStale() { - return blockContentsStale; - } - - public void markStaleAfterFailover() { - heartbeatedSinceFailover = false; - blockContentsStale = true; - } - - public void receivedBlockReport() { - if (heartbeatedSinceFailover) { - blockContentsStale = false; - } - firstBlockReport = false; - } - - boolean isFirstBlockReport() { - return firstBlockReport; - } - @Override public String dumpDatanode() { StringBuilder sb = new StringBuilder(super.dumpDatanode()); @@ -606,4 +562,18 @@ public class DatanodeDescriptor extends DatanodeInfo { } return sb.toString(); } + + DatanodeStorageInfo updateStorage(DatanodeStorage s) { + synchronized (storageMap) { + DatanodeStorageInfo storage = storageMap.get(s.getStorageID()); + if (storage == null) { + LOG.info("Adding new storage ID " + s.getStorageID() + + " for DN " + getXferAddr()); + storage = new DatanodeStorageInfo(this, s); + storageMap.put(s.getStorageID(), storage); + } + return storage; + } + } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index a8d775e9b0c..9352afafc80 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -403,9 +403,13 @@ public class DatanodeManager { } - /** Get a datanode descriptor given corresponding storageID */ - DatanodeDescriptor getDatanode(final String storageID) { - return datanodeMap.get(storageID); + /** Get a datanode descriptor given corresponding DatanodeUUID */ + DatanodeDescriptor getDatanode(final String datanodeUuid) { + if (datanodeUuid == null) { + return null; + } + + return datanodeMap.get(datanodeUuid); } /** @@ -417,7 +421,7 @@ public class DatanodeManager { */ public DatanodeDescriptor getDatanode(DatanodeID nodeID ) throws UnregisteredNodeException { - final DatanodeDescriptor node = getDatanode(nodeID.getStorageID()); + final DatanodeDescriptor node = getDatanode(nodeID.getDatanodeUuid()); if (node == null) return null; if (!node.getXferAddr().equals(nodeID.getXferAddr())) { @@ -430,6 +434,20 @@ public class DatanodeManager { return node; } + public DatanodeStorageInfo[] getDatanodeStorageInfos( + DatanodeID[] datanodeID, String[] storageIDs) + throws UnregisteredNodeException { + if (datanodeID.length == 0) { + return null; + } + final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length]; + for(int i = 0; i < datanodeID.length; i++) { + final DatanodeDescriptor dd = getDatanode(datanodeID[i]); + storages[i] = dd.getStorageInfo(storageIDs[i]); + } + return storages; + } + /** Prints information about all datanodes. */ void datanodeDump(final PrintWriter out) { synchronized (datanodeMap) { @@ -507,7 +525,7 @@ public class DatanodeManager { // remove from host2DatanodeMap the datanodeDescriptor removed // from datanodeMap before adding node to host2DatanodeMap. synchronized(datanodeMap) { - host2DatanodeMap.remove(datanodeMap.put(node.getStorageID(), node)); + host2DatanodeMap.remove(datanodeMap.put(node.getDatanodeUuid(), node)); } networktopology.add(node); // may throw InvalidTopologyException @@ -522,7 +540,7 @@ public class DatanodeManager { /** Physically remove node from datanodeMap. */ private void wipeDatanode(final DatanodeID node) { - final String key = node.getStorageID(); + final String key = node.getDatanodeUuid(); synchronized (datanodeMap) { host2DatanodeMap.remove(datanodeMap.remove(key)); } @@ -683,8 +701,10 @@ public class DatanodeManager { /** Start decommissioning the specified datanode. */ private void startDecommission(DatanodeDescriptor node) { if (!node.isDecommissionInProgress() && !node.isDecommissioned()) { - LOG.info("Start Decommissioning " + node + " with " + - node.numBlocks() + " blocks"); + for (DatanodeStorageInfo storage : node.getStorageInfos()) { + LOG.info("Start Decommissioning " + node + " " + storage + + " with " + storage.numBlocks() + " blocks"); + } heartbeatManager.startDecommission(node); node.decommissioningStatus.setStartTime(now()); @@ -706,24 +726,6 @@ public class DatanodeManager { } } - /** - * Generate new storage ID. - * - * @return unique storage ID - * - * Note: that collisions are still possible if somebody will try - * to bring in a data storage from a different cluster. - */ - private String newStorageID() { - String newID = null; - while(newID == null) { - newID = "DS" + Integer.toString(DFSUtil.getRandom().nextInt()); - if (datanodeMap.get(newID) != null) - newID = null; - } - return newID; - } - /** * Register the given datanode with the namenode. NB: the given * registration is mutated and given back to the datanode. @@ -762,9 +764,9 @@ public class DatanodeManager { } NameNode.stateChangeLog.info("BLOCK* registerDatanode: from " - + nodeReg + " storage " + nodeReg.getStorageID()); + + nodeReg + " storage " + nodeReg.getDatanodeUuid()); - DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID()); + DatanodeDescriptor nodeS = getDatanode(nodeReg.getDatanodeUuid()); DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr( nodeReg.getIpAddr(), nodeReg.getXferPort()); @@ -799,7 +801,7 @@ public class DatanodeManager { */ NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS + " is replaced by " + nodeReg + " with the same storageID " - + nodeReg.getStorageID()); + + nodeReg.getDatanodeUuid()); } boolean success = false; @@ -831,20 +833,8 @@ public class DatanodeManager { } } return; - } - - // this is a new datanode serving a new data storage - if ("".equals(nodeReg.getStorageID())) { - // this data storage has never been registered - // it is either empty or was created by pre-storageID version of DFS - nodeReg.setStorageID(newStorageID()); - if (NameNode.stateChangeLog.isDebugEnabled()) { - NameNode.stateChangeLog.debug( - "BLOCK* NameSystem.registerDatanode: " - + "new storageID " + nodeReg.getStorageID() + " assigned."); - } } - + DatanodeDescriptor nodeDescr = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK); boolean success = false; @@ -1212,8 +1202,7 @@ public class DatanodeManager { /** Handle heartbeat from datanodes. */ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, - final String blockPoolId, - long capacity, long dfsUsed, long remaining, long blockPoolUsed, + StorageReport[] reports, final String blockPoolId, int xceiverCount, int maxTransfers, int failedVolumes ) throws IOException { synchronized (heartbeatManager) { @@ -1235,8 +1224,8 @@ public class DatanodeManager { return new DatanodeCommand[]{RegisterCommand.REGISTER}; } - heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed, - remaining, blockPoolUsed, xceiverCount, failedVolumes); + heartbeatManager.updateHeartbeat(nodeinfo, reports, + xceiverCount, failedVolumes); // If we are in safemode, do not send back any recovery / replication // requests. Don't even drain the existing queue of work. @@ -1251,32 +1240,32 @@ public class DatanodeManager { BlockRecoveryCommand brCommand = new BlockRecoveryCommand( blocks.length); for (BlockInfoUnderConstruction b : blocks) { - DatanodeDescriptor[] expectedLocations = b.getExpectedLocations(); + final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations(); // Skip stale nodes during recovery - not heart beated for some time (30s by default). - List recoveryLocations = - new ArrayList(expectedLocations.length); - for (int i = 0; i < expectedLocations.length; i++) { - if (!expectedLocations[i].isStale(this.staleInterval)) { - recoveryLocations.add(expectedLocations[i]); + final List recoveryLocations = + new ArrayList(storages.length); + for (int i = 0; i < storages.length; i++) { + if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) { + recoveryLocations.add(storages[i]); } } // If we only get 1 replica after eliminating stale nodes, then choose all // replicas for recovery and let the primary data node handle failures. if (recoveryLocations.size() > 1) { - if (recoveryLocations.size() != expectedLocations.length) { + if (recoveryLocations.size() != storages.length) { LOG.info("Skipped stale nodes for recovery : " + - (expectedLocations.length - recoveryLocations.size())); + (storages.length - recoveryLocations.size())); } brCommand.add(new RecoveringBlock( new ExtendedBlock(blockPoolId, b), - recoveryLocations.toArray(new DatanodeDescriptor[recoveryLocations.size()]), + DatanodeStorageInfo.toDatanodeInfos(recoveryLocations), b.getBlockRecoveryId())); } else { // If too many replicas are stale, then choose all replicas to participate // in block recovery. brCommand.add(new RecoveringBlock( new ExtendedBlock(blockPoolId, b), - expectedLocations, + DatanodeStorageInfo.toDatanodeInfos(storages), b.getBlockRecoveryId())); } } @@ -1342,7 +1331,9 @@ public class DatanodeManager { LOG.info("Marking all datandoes as stale"); synchronized (datanodeMap) { for (DatanodeDescriptor dn : datanodeMap.values()) { - dn.markStaleAfterFailover(); + for(DatanodeStorageInfo storage : dn.getStorageInfos()) { + storage.markStaleAfterFailover(); + } } } } @@ -1365,3 +1356,4 @@ public class DatanodeManager { return getClass().getSimpleName() + ": " + host2DatanodeMap; } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java new file mode 100644 index 00000000000..914d58a93ef --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java @@ -0,0 +1,288 @@ +/** + * 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.blockmanagement; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hdfs.StorageType; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; + +/** + * A Datanode has one or more storages. A storage in the Datanode is represented + * by this class. + */ +public class DatanodeStorageInfo { + public static final DatanodeStorageInfo[] EMPTY_ARRAY = {}; + + public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) { + return toDatanodeInfos(Arrays.asList(storages)); + } + static DatanodeInfo[] toDatanodeInfos(List storages) { + final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()]; + for(int i = 0; i < storages.size(); i++) { + datanodes[i] = storages.get(i).getDatanodeDescriptor(); + } + return datanodes; + } + + static DatanodeDescriptor[] toDatanodeDescriptors( + DatanodeStorageInfo[] storages) { + DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length]; + for (int i = 0; i < storages.length; ++i) { + datanodes[i] = storages[i].getDatanodeDescriptor(); + } + return datanodes; + } + + public static String[] toStorageIDs(DatanodeStorageInfo[] storages) { + String[] storageIDs = new String[storages.length]; + for(int i = 0; i < storageIDs.length; i++) { + storageIDs[i] = storages[i].getStorageID(); + } + return storageIDs; + } + + public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) { + StorageType[] storageTypes = new StorageType[storages.length]; + for(int i = 0; i < storageTypes.length; i++) { + storageTypes[i] = storages[i].getStorageType(); + } + return storageTypes; + } + + /** + * Iterates over the list of blocks belonging to the data-node. + */ + class BlockIterator implements Iterator { + private BlockInfo current; + + BlockIterator(BlockInfo head) { + this.current = head; + } + + public boolean hasNext() { + return current != null; + } + + public BlockInfo next() { + BlockInfo res = current; + current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this)); + return res; + } + + public void remove() { + throw new UnsupportedOperationException("Sorry. can't remove."); + } + } + + private final DatanodeDescriptor dn; + private final String storageID; + private final StorageType storageType; + private final State state; + + private long capacity; + private long dfsUsed; + private long remaining; + private long blockPoolUsed; + + private volatile BlockInfo blockList = null; + private int numBlocks = 0; + + /** The number of block reports received */ + private int blockReportCount = 0; + + /** + * Set to false on any NN failover, and reset to true + * whenever a block report is received. + */ + private boolean heartbeatedSinceFailover = false; + + /** + * At startup or at failover, the storages in the cluster may have pending + * block deletions from a previous incarnation of the NameNode. The block + * contents are considered as stale until a block report is received. When a + * storage is considered as stale, the replicas on it are also considered as + * stale. If any block has at least one stale replica, then no invalidations + * will be processed for this block. See HDFS-1972. + */ + private boolean blockContentsStale = true; + + DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) { + this.dn = dn; + this.storageID = s.getStorageID(); + this.storageType = s.getStorageType(); + this.state = s.getState(); + } + + int getBlockReportCount() { + return blockReportCount; + } + + void setBlockReportCount(int blockReportCount) { + this.blockReportCount = blockReportCount; + } + + boolean areBlockContentsStale() { + return blockContentsStale; + } + + void markStaleAfterFailover() { + heartbeatedSinceFailover = false; + blockContentsStale = true; + } + + void receivedHeartbeat(StorageReport report) { + updateState(report); + heartbeatedSinceFailover = true; + } + + void receivedBlockReport() { + if (heartbeatedSinceFailover) { + blockContentsStale = false; + } + blockReportCount++; + } + + @VisibleForTesting + public void setUtilizationForTesting(long capacity, long dfsUsed, + long remaining, long blockPoolUsed) { + this.capacity = capacity; + this.dfsUsed = dfsUsed; + this.remaining = remaining; + this.blockPoolUsed = blockPoolUsed; + } + + State getState() { + return this.state; + } + + String getStorageID() { + return storageID; + } + + StorageType getStorageType() { + return storageType; + } + + long getCapacity() { + return capacity; + } + + long getDfsUsed() { + return dfsUsed; + } + + long getRemaining() { + return remaining; + } + + long getBlockPoolUsed() { + return blockPoolUsed; + } + + boolean addBlock(BlockInfo b) { + if(!b.addStorage(this)) + return false; + // add to the head of the data-node list + blockList = b.listInsert(blockList, this); + numBlocks++; + return true; + } + + boolean removeBlock(BlockInfo b) { + blockList = b.listRemove(blockList, this); + if (b.removeStorage(this)) { + numBlocks--; + return true; + } else { + return false; + } + } + + int numBlocks() { + return numBlocks; + } + + Iterator getBlockIterator() { + return new BlockIterator(blockList); + + } + + /** + * Move block to the head of the list of blocks belonging to the data-node. + * @return the index of the head of the blockList + */ + int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) { + blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex); + return curIndex; + } + + /** + * Used for testing only + * @return the head of the blockList + */ + @VisibleForTesting + BlockInfo getBlockListHeadForTesting(){ + return blockList; + } + + void updateState(StorageReport r) { + capacity = r.getCapacity(); + dfsUsed = r.getDfsUsed(); + remaining = r.getRemaining(); + blockPoolUsed = r.getBlockPoolUsed(); + } + + public DatanodeDescriptor getDatanodeDescriptor() { + return dn; + } + + /** Increment the number of blocks scheduled for each given storage */ + public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) { + for (DatanodeStorageInfo s : storages) { + s.getDatanodeDescriptor().incrementBlocksScheduled(); + } + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj == null || !(obj instanceof DatanodeStorageInfo)) { + return false; + } + final DatanodeStorageInfo that = (DatanodeStorageInfo)obj; + return this.storageID.equals(that.storageID); + } + + @Override + public int hashCode() { + return storageID.hashCode(); + } + + @Override + public String toString() { + return "[" + storageType + "]" + storageID + ":" + state; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java index 0bff1bf52f7..1f146d5e252 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.server.namenode.Namesystem; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Time; @@ -170,7 +171,7 @@ class HeartbeatManager implements DatanodeStatistics { addDatanode(d); //update its timestamp - d.updateHeartbeat(0L, 0L, 0L, 0L, 0, 0); + d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0); } } @@ -192,11 +193,9 @@ class HeartbeatManager implements DatanodeStatistics { } synchronized void updateHeartbeat(final DatanodeDescriptor node, - long capacity, long dfsUsed, long remaining, long blockPoolUsed, - int xceiverCount, int failedVolumes) { + StorageReport[] reports, int xceiverCount, int failedVolumes) { stats.subtract(node); - node.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed, - xceiverCount, failedVolumes); + node.updateHeartbeat(reports, xceiverCount, failedVolumes); stats.add(node); } @@ -341,3 +340,4 @@ class HeartbeatManager implements DatanodeStatistics { } } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java index d4bf76f95b8..f0d5d088dd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java @@ -80,10 +80,10 @@ class InvalidateBlocks { */ synchronized void add(final Block block, final DatanodeInfo datanode, final boolean log) { - LightWeightHashSet set = node2blocks.get(datanode.getStorageID()); + LightWeightHashSet set = node2blocks.get(datanode.getDatanodeUuid()); if (set == null) { set = new LightWeightHashSet(); - node2blocks.put(datanode.getStorageID(), set); + node2blocks.put(datanode.getDatanodeUuid(), set); } if (set.add(block)) { numBlocks++; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java index 41975d3371d..5d63fefaaf5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java @@ -34,5 +34,5 @@ public interface MutableBlockCollection extends BlockCollection { * and set the locations. */ public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock, - DatanodeDescriptor[] locations) throws IOException; + DatanodeStorageInfo[] storages) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java index 860d1d261f8..8afd3ce6ba7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java @@ -42,11 +42,13 @@ class PendingDataNodeMessages { static class ReportedBlockInfo { private final Block block; private final DatanodeDescriptor dn; + private final String storageID; private final ReplicaState reportedState; - ReportedBlockInfo(DatanodeDescriptor dn, Block block, + ReportedBlockInfo(DatanodeDescriptor dn, String storageID, Block block, ReplicaState reportedState) { this.dn = dn; + this.storageID = storageID; this.block = block; this.reportedState = reportedState; } @@ -58,6 +60,10 @@ class PendingDataNodeMessages { DatanodeDescriptor getNode() { return dn; } + + String getStorageID() { + return storageID; + } ReplicaState getReportedState() { return reportedState; @@ -70,11 +76,11 @@ class PendingDataNodeMessages { } } - void enqueueReportedBlock(DatanodeDescriptor dn, Block block, + void enqueueReportedBlock(DatanodeDescriptor dn, String storageID, Block block, ReplicaState reportedState) { block = new Block(block); getBlockQueue(block).add( - new ReportedBlockInfo(dn, block, reportedState)); + new ReportedBlockInfo(dn, storageID, block, reportedState)); count++; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java index 0889f66ab08..ce53bdb2e0e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java @@ -236,6 +236,8 @@ public abstract class Storage extends StorageInfo { final boolean useLock; // flag to enable storage lock final StorageDirType dirType; // storage dir type FileLock lock; // storage lock + + private String storageUuid = null; // Storage directory identifier. public StorageDirectory(File dir) { // default dirType is null @@ -246,6 +248,14 @@ public abstract class Storage extends StorageInfo { this(dir, dirType, true); } + public void setStorageUuid(String storageUuid) { + this.storageUuid = storageUuid; + } + + public String getStorageUuid() { + return storageUuid; + } + /** * Constructor * @param dir directory corresponding to the storage diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java index 4ca92258791..29300a8c3ae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java @@ -27,6 +27,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -159,31 +160,32 @@ class BPOfferService { synchronized NamespaceInfo getNamespaceInfo() { return bpNSInfo; } - + @Override public synchronized String toString() { if (bpNSInfo == null) { // If we haven't yet connected to our NN, we don't yet know our // own block pool ID. // If _none_ of the block pools have connected yet, we don't even - // know the storage ID of this DN. - String storageId = dn.getStorageId(); - if (storageId == null || "".equals(storageId)) { - storageId = "unknown"; + // know the DatanodeID ID of this DN. + String datanodeUuid = dn.getDatanodeUuid(); + + if (datanodeUuid == null || datanodeUuid.isEmpty()) { + datanodeUuid = "unassigned"; } - return "Block pool (storage id " + storageId + - ")"; + return "Block pool (Datanode Uuid " + datanodeUuid + ")"; } else { return "Block pool " + getBlockPoolId() + - " (storage id " + dn.getStorageId() + - ")"; + " (Datanode Uuid " + dn.getDatanodeUuid() + + ")"; } } - void reportBadBlocks(ExtendedBlock block) { + void reportBadBlocks(ExtendedBlock block, + String storageUuid, StorageType storageType) { checkBlock(block); for (BPServiceActor actor : bpServices) { - actor.reportBadBlocks(block); + actor.reportBadBlocks(block, storageUuid, storageType); } } @@ -192,7 +194,8 @@ class BPOfferService { * till namenode is informed before responding with success to the * client? For now we don't. */ - void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) { + void notifyNamenodeReceivedBlock( + ExtendedBlock block, String delHint, String storageUuid) { checkBlock(block); checkDelHint(delHint); ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo( @@ -201,7 +204,7 @@ class BPOfferService { delHint); for (BPServiceActor actor : bpServices) { - actor.notifyNamenodeBlockImmediately(bInfo); + actor.notifyNamenodeBlockImmediately(bInfo, storageUuid); } } @@ -218,23 +221,23 @@ class BPOfferService { "delHint is null"); } - void notifyNamenodeDeletedBlock(ExtendedBlock block) { + void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) { checkBlock(block); ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo( block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null); for (BPServiceActor actor : bpServices) { - actor.notifyNamenodeDeletedBlock(bInfo); + actor.notifyNamenodeDeletedBlock(bInfo, storageUuid); } } - void notifyNamenodeReceivingBlock(ExtendedBlock block) { + void notifyNamenodeReceivingBlock(ExtendedBlock block, String storageUuid) { checkBlock(block); ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo( block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null); for (BPServiceActor actor : bpServices) { - actor.notifyNamenodeBlockImmediately(bInfo); + actor.notifyNamenodeBlockImmediately(bInfo, storageUuid); } } @@ -337,7 +340,7 @@ class BPOfferService { } } - synchronized DatanodeRegistration createRegistration() { + synchronized DatanodeRegistration createRegistration() throws IOException { Preconditions.checkState(bpNSInfo != null, "getRegistration() can only be called after initial handshake"); return dn.createBPRegistration(bpNSInfo); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index faa0c78da97..5e2250915d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -22,7 +22,6 @@ import static org.apache.hadoop.util.Time.now; import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketTimeoutException; -import java.net.URI; import java.util.Collection; import java.util.Map; @@ -30,6 +29,7 @@ import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.hdfs.DFSUtil; +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; @@ -51,7 +51,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionUtil; @@ -97,9 +96,9 @@ class BPServiceActor implements Runnable { * keyed by block ID, contains the pending changes which have yet to be * reported to the NN. Access should be synchronized on this object. */ - private final Map pendingIncrementalBR - = Maps.newHashMap(); - + private final Map + pendingIncrementalBRperStorage = Maps.newHashMap(); + private volatile int pendingReceivedRequests = 0; private volatile boolean shouldServiceRun = true; private final DataNode dn; @@ -241,12 +240,15 @@ class BPServiceActor implements Runnable { resetBlockReportTime = true; // reset future BRs for randomness } - void reportBadBlocks(ExtendedBlock block) { + void reportBadBlocks(ExtendedBlock block, + String storageUuid, StorageType storageType) { if (bpRegistration == null) { return; } DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) }; - LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) }; + String[] uuids = { storageUuid }; + StorageType[] types = { storageType }; + LocatedBlock[] blocks = { new LocatedBlock(block, dnArr, uuids, types) }; try { bpNamenode.reportBadBlocks(blocks); @@ -260,49 +262,100 @@ class BPServiceActor implements Runnable { } /** - * Report received blocks and delete hints to the Namenode - * + * Report received blocks and delete hints to the Namenode for each + * storage. + * * @throws IOException */ private void reportReceivedDeletedBlocks() throws IOException { - // check if there are newly received blocks - ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null; - synchronized (pendingIncrementalBR) { - int numBlocks = pendingIncrementalBR.size(); - if (numBlocks > 0) { - // - // Send newly-received and deleted blockids to namenode - // - receivedAndDeletedBlockArray = pendingIncrementalBR - .values().toArray(new ReceivedDeletedBlockInfo[numBlocks]); - } - pendingIncrementalBR.clear(); - } - if (receivedAndDeletedBlockArray != null) { - StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks( - bpRegistration.getStorageID(), receivedAndDeletedBlockArray) }; - boolean success = false; - try { - bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(), - report); - success = true; - } finally { - synchronized (pendingIncrementalBR) { - if (!success) { - // If we didn't succeed in sending the report, put all of the - // blocks back onto our queue, but only in the case where we didn't - // put something newer in the meantime. - for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) { - if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) { - pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi); - } - } - } - pendingReceivedRequests = pendingIncrementalBR.size(); + // Generate a list of the pending reports for each storage under the lock + Map blockArrays = Maps.newHashMap(); + synchronized (pendingIncrementalBRperStorage) { + for (Map.Entry entry : + pendingIncrementalBRperStorage.entrySet()) { + final String storageUuid = entry.getKey(); + final PerStoragePendingIncrementalBR perStorageMap = entry.getValue(); + + if (perStorageMap.getBlockInfoCount() > 0) { + // Send newly-received and deleted blockids to namenode + ReceivedDeletedBlockInfo[] rdbi = perStorageMap.dequeueBlockInfos(); + pendingReceivedRequests = + (pendingReceivedRequests > rdbi.length ? + (pendingReceivedRequests - rdbi.length) : 0); + blockArrays.put(storageUuid, rdbi); } } } + + // Send incremental block reports to the Namenode outside the lock + for (Map.Entry entry : + blockArrays.entrySet()) { + final String storageUuid = entry.getKey(); + final ReceivedDeletedBlockInfo[] rdbi = entry.getValue(); + + StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks( + storageUuid, rdbi) }; + boolean success = false; + try { + bpNamenode.blockReceivedAndDeleted(bpRegistration, + bpos.getBlockPoolId(), report); + success = true; + } finally { + if (!success) { + synchronized (pendingIncrementalBRperStorage) { + // If we didn't succeed in sending the report, put all of the + // blocks back onto our queue, but only in the case where we + // didn't put something newer in the meantime. + PerStoragePendingIncrementalBR perStorageMap = + pendingIncrementalBRperStorage.get(storageUuid); + pendingReceivedRequests += perStorageMap.putMissingBlockInfos(rdbi); + } + } + } + } + } + + /** + * Retrieve the incremental BR state for a given storage UUID + * @param storageUuid + * @return + */ + private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage( + String storageUuid) { + PerStoragePendingIncrementalBR mapForStorage = + pendingIncrementalBRperStorage.get(storageUuid); + + if (mapForStorage == null) { + // This is the first time we are adding incremental BR state for + // this storage so create a new map. This is required once per + // storage, per service actor. + mapForStorage = new PerStoragePendingIncrementalBR(); + pendingIncrementalBRperStorage.put(storageUuid, mapForStorage); + } + + return mapForStorage; + } + + /** + * Add a blockInfo for notification to NameNode. If another entry + * exists for the same block it is removed. + * + * Caller must synchronize access using pendingIncrementalBRperStorage. + * @param bInfo + * @param storageUuid + */ + void addPendingReplicationBlockInfo(ReceivedDeletedBlockInfo bInfo, + String storageUuid) { + // Make sure another entry for the same block is first removed. + // There may only be one such entry. + for (Map.Entry entry : + pendingIncrementalBRperStorage.entrySet()) { + if (entry.getValue().removeBlockInfo(bInfo)) { + break; + } + } + getIncrementalBRMapForStorage(storageUuid).putBlockInfo(bInfo); } /* @@ -310,19 +363,19 @@ class BPServiceActor implements Runnable { * till namenode is informed before responding with success to the * client? For now we don't. */ - void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) { - synchronized (pendingIncrementalBR) { - pendingIncrementalBR.put( - bInfo.getBlock().getBlockId(), bInfo); + void notifyNamenodeBlockImmediately( + ReceivedDeletedBlockInfo bInfo, String storageUuid) { + synchronized (pendingIncrementalBRperStorage) { + addPendingReplicationBlockInfo(bInfo, storageUuid); pendingReceivedRequests++; - pendingIncrementalBR.notifyAll(); + pendingIncrementalBRperStorage.notifyAll(); } } - void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) { - synchronized (pendingIncrementalBR) { - pendingIncrementalBR.put( - bInfo.getBlock().getBlockId(), bInfo); + void notifyNamenodeDeletedBlock( + ReceivedDeletedBlockInfo bInfo, String storageUuid) { + synchronized (pendingIncrementalBRperStorage) { + addPendingReplicationBlockInfo(bInfo, storageUuid); } } @@ -331,13 +384,13 @@ class BPServiceActor implements Runnable { */ @VisibleForTesting void triggerBlockReportForTests() { - synchronized (pendingIncrementalBR) { + synchronized (pendingIncrementalBRperStorage) { lastBlockReport = 0; lastHeartbeat = 0; - pendingIncrementalBR.notifyAll(); + pendingIncrementalBRperStorage.notifyAll(); while (lastBlockReport == 0) { try { - pendingIncrementalBR.wait(100); + pendingIncrementalBRperStorage.wait(100); } catch (InterruptedException e) { return; } @@ -347,12 +400,12 @@ class BPServiceActor implements Runnable { @VisibleForTesting void triggerHeartbeatForTests() { - synchronized (pendingIncrementalBR) { + synchronized (pendingIncrementalBRperStorage) { lastHeartbeat = 0; - pendingIncrementalBR.notifyAll(); + pendingIncrementalBRperStorage.notifyAll(); while (lastHeartbeat == 0) { try { - pendingIncrementalBR.wait(100); + pendingIncrementalBRperStorage.wait(100); } catch (InterruptedException e) { return; } @@ -362,13 +415,13 @@ class BPServiceActor implements Runnable { @VisibleForTesting void triggerDeletionReportForTests() { - synchronized (pendingIncrementalBR) { + synchronized (pendingIncrementalBRperStorage) { lastDeletedReport = 0; - pendingIncrementalBR.notifyAll(); + pendingIncrementalBRperStorage.notifyAll(); while (lastDeletedReport == 0) { try { - pendingIncrementalBR.wait(100); + pendingIncrementalBRperStorage.wait(100); } catch (InterruptedException e) { return; } @@ -392,23 +445,38 @@ class BPServiceActor implements Runnable { // a FINALIZED one. reportReceivedDeletedBlocks(); + // Send one block report per known storage. + // Create block report long brCreateStartTime = now(); - BlockListAsLongs bReport = dn.getFSDataset().getBlockReport( - bpos.getBlockPoolId()); + long totalBlockCount = 0; + + Map perVolumeBlockLists = + dn.getFSDataset().getBlockReports(bpos.getBlockPoolId()); // Send block report long brSendStartTime = now(); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(bpRegistration.getStorageID()), - bReport.getBlockListAsLongs()) }; - cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report); + StorageBlockReport[] reports = + new StorageBlockReport[perVolumeBlockLists.size()]; + + int i = 0; + for(Map.Entry kvPair : perVolumeBlockLists.entrySet()) { + DatanodeStorage dnStorage = kvPair.getKey(); + BlockListAsLongs blockList = kvPair.getValue(); + totalBlockCount += blockList.getNumberOfBlocks(); + + reports[i++] = + new StorageBlockReport( + dnStorage, blockList.getBlockListAsLongs()); + } + + cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), reports); // Log the block report processing stats from Datanode perspective long brSendCost = now() - brSendStartTime; long brCreateCost = brSendStartTime - brCreateStartTime; dn.getMetrics().addBlockReport(brSendCost); - LOG.info("BlockReport of " + bReport.getNumberOfBlocks() + LOG.info("BlockReport of " + totalBlockCount + " blocks took " + brCreateCost + " msec to generate and " + brSendCost + " msecs for RPC and NN processing"); @@ -434,17 +502,15 @@ class BPServiceActor implements Runnable { HeartbeatResponse sendHeartBeat() throws IOException { + StorageReport[] reports = + dn.getFSDataset().getStorageReports(bpos.getBlockPoolId()); if (LOG.isDebugEnabled()) { - LOG.debug("Sending heartbeat from service actor: " + this); + LOG.debug("Sending heartbeat with " + reports.length + + " storage reports from service actor: " + this); } - // reports number of failed volumes - StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(), - false, - dn.getFSDataset().getCapacity(), - dn.getFSDataset().getDfsUsed(), - dn.getFSDataset().getRemaining(), - dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) }; - return bpNamenode.sendHeartbeat(bpRegistration, report, + + return bpNamenode.sendHeartbeat(bpRegistration, + reports, dn.getXmitsInProgress(), dn.getXceiverCount(), dn.getFSDataset().getNumFailedVolumes()); @@ -462,9 +528,9 @@ class BPServiceActor implements Runnable { } private String formatThreadName() { - Collection dataDirs = DataNode.getStorageDirs(dn.getConf()); - return "DataNode: [" + - StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " + + Collection dataDirs = + DataNode.getStorageLocations(dn.getConf()); + return "DataNode: [" + dataDirs.toString() + "] " + " heartbeating to " + nnAddr; } @@ -570,10 +636,10 @@ class BPServiceActor implements Runnable { // long waitTime = dnConf.heartBeatInterval - (Time.now() - lastHeartbeat); - synchronized(pendingIncrementalBR) { + synchronized(pendingIncrementalBRperStorage) { if (waitTime > 0 && pendingReceivedRequests == 0) { try { - pendingIncrementalBR.wait(waitTime); + pendingIncrementalBRperStorage.wait(waitTime); } catch (InterruptedException ie) { LOG.warn("BPOfferService for " + this + " interrupted"); } @@ -744,4 +810,68 @@ class BPServiceActor implements Runnable { } } + private static class PerStoragePendingIncrementalBR { + private Map pendingIncrementalBR = + Maps.newHashMap(); + + /** + * Return the number of blocks on this storage that have pending + * incremental block reports. + * @return + */ + int getBlockInfoCount() { + return pendingIncrementalBR.size(); + } + + /** + * Dequeue and return all pending incremental block report state. + * @return + */ + ReceivedDeletedBlockInfo[] dequeueBlockInfos() { + ReceivedDeletedBlockInfo[] blockInfos = + pendingIncrementalBR.values().toArray( + new ReceivedDeletedBlockInfo[getBlockInfoCount()]); + + pendingIncrementalBR.clear(); + return blockInfos; + } + + /** + * Add blocks from blockArray to pendingIncrementalBR, unless the + * block already exists in pendingIncrementalBR. + * @param blockArray list of blocks to add. + * @return the number of missing blocks that we added. + */ + int putMissingBlockInfos(ReceivedDeletedBlockInfo[] blockArray) { + int blocksPut = 0; + for (ReceivedDeletedBlockInfo rdbi : blockArray) { + if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) { + pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi); + ++blocksPut; + } + } + return blocksPut; + } + + /** + * Add pending incremental block report for a single block. + * @param blockID + * @param blockInfo + */ + void putBlockInfo(ReceivedDeletedBlockInfo blockInfo) { + pendingIncrementalBR.put(blockInfo.getBlock().getBlockId(), blockInfo); + } + + /** + * Remove pending incremental block report for a single block if it + * exists. + * + * @param blockInfo + * @return true if a report was removed, false if no report existed for + * the given block. + */ + boolean removeBlockInfo(ReceivedDeletedBlockInfo blockInfo) { + return (pendingIncrementalBR.remove(blockInfo.getBlock().getBlockId()) != null); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java index 32f9d9ea3e9..13a83bce5fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java @@ -187,7 +187,7 @@ class BlockPoolSliceScanner { + hours + " hours for block pool " + bpid); // get the list of blocks and arrange them in random order - List arr = dataset.getFinalizedBlocks(blockPoolId); + List arr = dataset.getFinalizedBlocks(blockPoolId); Collections.shuffle(arr); long scanTime = -1; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 667874978fa..eb91432a822 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -162,7 +162,8 @@ class BlockReceiver implements Closeable { switch (stage) { case PIPELINE_SETUP_CREATE: replicaInfo = datanode.data.createRbw(block); - datanode.notifyNamenodeReceivingBlock(block); + datanode.notifyNamenodeReceivingBlock( + block, replicaInfo.getStorageUuid()); break; case PIPELINE_SETUP_STREAMING_RECOVERY: replicaInfo = datanode.data.recoverRbw( @@ -176,7 +177,8 @@ class BlockReceiver implements Closeable { block.getLocalBlock()); } block.setGenerationStamp(newGs); - datanode.notifyNamenodeReceivingBlock(block); + datanode.notifyNamenodeReceivingBlock( + block, replicaInfo.getStorageUuid()); break; case PIPELINE_SETUP_APPEND_RECOVERY: replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd); @@ -185,7 +187,8 @@ class BlockReceiver implements Closeable { block.getLocalBlock()); } block.setGenerationStamp(newGs); - datanode.notifyNamenodeReceivingBlock(block); + datanode.notifyNamenodeReceivingBlock( + block, replicaInfo.getStorageUuid()); break; case TRANSFER_RBW: case TRANSFER_FINALIZED: @@ -252,6 +255,10 @@ class BlockReceiver implements Closeable { /** Return the datanode object. */ DataNode getDataNode() {return datanode;} + String getStorageUuid() { + return replicaInfo.getStorageUuid(); + } + /** * close files. */ @@ -1027,7 +1034,7 @@ class BlockReceiver implements Closeable { final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; block.setNumBytes(replicaInfo.getNumBytes()); datanode.data.finalizeBlock(block); - datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT); + datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, replicaInfo.getStorageUuid()); if (ClientTraceLog.isInfoEnabled() && isClient) { long offset = 0; DatanodeRegistration dnR = @@ -1035,7 +1042,7 @@ class BlockReceiver implements Closeable { ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT, inAddr, myAddr, block.getNumBytes(), "HDFS_WRITE", clientname, offset, - dnR.getStorageID(), block, endTime-startTime)); + dnR.getDatanodeUuid(), block, endTime-startTime)); } else { LOG.info("Received " + block + " size " + block.getNumBytes() + " from " + inAddr); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index f4f519f8f50..bbab2fc18c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -17,10 +17,40 @@ */ package org.apache.hadoop.hdfs.server.datanode; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.protobuf.BlockingService; +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.SocketChannel; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.management.ObjectName; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -39,21 +69,42 @@ 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.*; -import org.apache.hadoop.hdfs.protocol.datatransfer.*; +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.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.*; -import org.apache.hadoop.hdfs.security.token.block.*; +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.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; import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.StorageInfo; -import org.apache.hadoop.hdfs.server.common.Util; import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; @@ -62,7 +113,11 @@ 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.*; +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.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.resources.Param; import org.apache.hadoop.http.HttpConfig; @@ -84,23 +139,21 @@ 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.*; +import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.DiskChecker; 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 java.io.*; -import java.net.*; -import java.nio.channels.ClosedByInterruptException; -import java.nio.channels.SocketChannel; -import java.security.PrivilegedExceptionAction; -import java.util.*; -import java.util.concurrent.atomic.AtomicInteger; - -import javax.management.ObjectName; - -import static org.apache.hadoop.hdfs.DFSConfigKeys.*; -import static org.apache.hadoop.util.ExitUtil.terminate; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.protobuf.BlockingService; /********************************************************** * DataNode is a class (and program) that stores a set of @@ -209,7 +262,7 @@ public class DataNode extends Configured private JvmPauseMonitor pauseMonitor; private SecureResources secureResources = null; - private AbstractList dataDirs; + private List dataDirs; private Configuration conf; private final List usersWithLocalPathAccess; @@ -218,21 +271,12 @@ public class DataNode extends Configured private final boolean getHdfsBlockLocationsEnabled; private ObjectName dataNodeInfoBeanName; - /** - * Create the DataNode given a configuration and an array of dataDirs. - * 'dataDirs' is where the blocks are stored. - */ - DataNode(final Configuration conf, - final AbstractList dataDirs) throws IOException { - this(conf, dataDirs, null); - } - /** * Create the DataNode given a configuration, an array of dataDirs, * and a namenode proxy */ - DataNode(final Configuration conf, - final AbstractList dataDirs, + DataNode(final Configuration conf, + final List dataDirs, final SecureResources resources) throws IOException { super(conf); @@ -491,7 +535,7 @@ public class DataNode extends Configured directoryScanner.start(); } else { LOG.info("Periodic Directory Tree Verification scan is disabled because " + - reason); + reason); } } @@ -563,10 +607,11 @@ public class DataNode extends Configured } // calls specific to BP - protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) { + protected void notifyNamenodeReceivedBlock( + ExtendedBlock block, String delHint, String storageUuid) { BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId()); if(bpos != null) { - bpos.notifyNamenodeReceivedBlock(block, delHint); + bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid); } else { LOG.error("Cannot find BPOfferService for reporting block received for bpid=" + block.getBlockPoolId()); @@ -574,10 +619,11 @@ public class DataNode extends Configured } // calls specific to BP - protected void notifyNamenodeReceivingBlock(ExtendedBlock block) { + protected void notifyNamenodeReceivingBlock( + ExtendedBlock block, String storageUuid) { BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId()); if(bpos != null) { - bpos.notifyNamenodeReceivingBlock(block); + bpos.notifyNamenodeReceivingBlock(block, storageUuid); } else { LOG.error("Cannot find BPOfferService for reporting block receiving for bpid=" + block.getBlockPoolId()); @@ -585,10 +631,10 @@ public class DataNode extends Configured } /** Notify the corresponding namenode to delete the block. */ - public void notifyNamenodeDeletedBlock(ExtendedBlock block) { + public void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) { BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId()); if (bpos != null) { - bpos.notifyNamenodeDeletedBlock(block); + bpos.notifyNamenodeDeletedBlock(block, storageUuid); } else { LOG.error("Cannot find BPOfferService for reporting block deleted for bpid=" + block.getBlockPoolId()); @@ -600,7 +646,9 @@ public class DataNode extends Configured */ public void reportBadBlocks(ExtendedBlock block) throws IOException{ BPOfferService bpos = getBPOSForBlock(block); - bpos.reportBadBlocks(block); + FsVolumeSpi volume = getFSDataset().getVolume(block); + bpos.reportBadBlocks( + block, volume.getStorageID(), volume.getStorageType()); } /** @@ -672,7 +720,7 @@ public class DataNode extends Configured * @throws IOException */ void startDataNode(Configuration conf, - AbstractList dataDirs, + List dataDirs, // DatanodeProtocol namenode, SecureResources resources ) throws IOException { @@ -712,19 +760,42 @@ public class DataNode extends Configured readaheadPool = ReadaheadPool.getInstance(); } + public static String generateUuid() { + return UUID.randomUUID().toString(); + } + + /** + * Verify that the DatanodeUuid has been initialized. If this is a new + * datanode then we generate a new Datanode Uuid and persist it to disk. + * + * @throws IOException + */ + private synchronized void checkDatanodeUuid() throws IOException { + if (storage.getDatanodeUuid() == null) { + storage.setDatanodeUuid(generateUuid()); + storage.writeAll(); + LOG.info("Generated and persisted new Datanode UUID " + + storage.getDatanodeUuid()); + } + } + /** * Create a DatanodeRegistration for a specific block pool. * @param nsInfo the namespace info from the first part of the NN handshake */ - DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) { + DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) + throws IOException { StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID()); if (storageInfo == null) { // it's null in the case of SimulatedDataSet storageInfo = new StorageInfo(nsInfo); } + + checkDatanodeUuid(); + DatanodeID dnId = new DatanodeID( streamingAddr.getAddress().getHostAddress(), hostName, - getStorageId(), getXferPort(), getInfoPort(), + storage.getDatanodeUuid(), getXferPort(), getInfoPort(), infoSecurePort, getIpcPort()); return new DatanodeRegistration(dnId, storageInfo, new ExportedBlockKeys(), VersionInfo.getVersion()); @@ -743,16 +814,10 @@ public class DataNode extends Configured id = bpRegistration; } - if (storage.getStorageID().equals("")) { - // This is a fresh datanode, persist the NN-provided storage ID - storage.setStorageID(bpRegistration.getStorageID()); - storage.writeAll(); - LOG.info("New storage id " + bpRegistration.getStorageID() - + " is assigned to data-node " + bpRegistration); - } else if(!storage.getStorageID().equals(bpRegistration.getStorageID())) { - throw new IOException("Inconsistent storage IDs. Name-node returned " - + bpRegistration.getStorageID() - + ". Expecting " + storage.getStorageID()); + if(!storage.getDatanodeUuid().equals(bpRegistration.getDatanodeUuid())) { + throw new IOException("Inconsistent Datanode IDs. Name-node returned " + + bpRegistration.getDatanodeUuid() + + ". Expecting " + storage.getDatanodeUuid()); } registerBlockPoolWithSecretManager(bpRegistration, blockPoolId); @@ -873,7 +938,7 @@ public class DataNode extends Configured final StorageInfo bpStorage = storage.getBPStorage(bpid); LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID() + ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion() - + ";nsInfo=" + nsInfo); + + ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid()); } synchronized(this) { @@ -900,10 +965,6 @@ public class DataNode extends Configured return streamingAddr.getPort(); } - String getStorageId() { - return storage.getStorageID(); - } - /** * @return name useful for logging */ @@ -989,34 +1050,6 @@ public class DataNode extends Configured return metrics; } - public static void setNewStorageID(DatanodeID dnId) { - LOG.info("Datanode is " + dnId); - dnId.setStorageID(createNewStorageId(dnId.getXferPort())); - } - - /** - * @return a unique storage ID of form "DS-randInt-ipaddr-port-timestamp" - */ - static String createNewStorageId(int port) { - // It is unlikely that we will create a non-unique storage ID - // for the following reasons: - // a) SecureRandom is a cryptographically strong random number generator - // b) IP addresses will likely differ on different hosts - // c) DataNode xfer ports will differ on the same host - // d) StorageIDs will likely be generated at different times (in ms) - // A conflict requires that all four conditions are violated. - // NB: The format of this string can be changed in the future without - // requiring that old SotrageIDs be updated. - String ip = "unknownIP"; - try { - ip = DNS.getDefaultIP("default"); - } catch (UnknownHostException ignored) { - LOG.warn("Could not find an IP address for the \"default\" inteface."); - } - int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE); - return "DS-" + rand + "-" + ip + "-" + port + "-" + Time.now(); - } - /** Ensure the authentication method is kerberos */ private void checkKerberosAuthMethod(String msg) throws IOException { // User invoking the call must be same as the datanode user @@ -1342,8 +1375,10 @@ public class DataNode extends Configured // Check if NN recorded length matches on-disk length long onDiskLength = data.getLength(block); if (block.getNumBytes() > onDiskLength) { + FsVolumeSpi volume = getFSDataset().getVolume(block); // Shorter on-disk len indicates corruption so report NN the corrupt block - bpos.reportBadBlocks(block); + bpos.reportBadBlocks( + block, volume.getStorageID(), volume.getStorageType()); LOG.warn("Can't replicate block " + block + " because on-disk length " + onDiskLength + " is shorter than NameNode recorded length " + block.getNumBytes()); @@ -1607,11 +1642,11 @@ public class DataNode extends Configured * @param block * @param delHint */ - void closeBlock(ExtendedBlock block, String delHint) { + void closeBlock(ExtendedBlock block, String delHint, String storageUuid) { metrics.incrBlocksWritten(); BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId()); if(bpos != null) { - bpos.notifyNamenodeReceivedBlock(block, delHint); + bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid); } else { LOG.warn("Cannot find BPOfferService for reporting block received for bpid=" + block.getBlockPoolId()); @@ -1675,17 +1710,32 @@ public class DataNode extends Configured printUsage(System.err); return null; } - Collection dataDirs = getStorageDirs(conf); + Collection dataLocations = getStorageLocations(conf); UserGroupInformation.setConfiguration(conf); SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY, DFS_DATANODE_USER_NAME_KEY); - return makeInstance(dataDirs, conf, resources); + return makeInstance(dataLocations, conf, resources); } - static Collection getStorageDirs(Configuration conf) { - Collection dirNames = - conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY); - return Util.stringCollectionAsURIs(dirNames); + public static List getStorageLocations(Configuration conf) { + Collection rawLocations = + conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY); + List locations = + new ArrayList(rawLocations.size()); + + for(String locationString : rawLocations) { + final StorageLocation location; + try { + location = StorageLocation.parse(locationString); + } catch (IOException ioe) { + throw new IllegalArgumentException("Failed to parse conf property " + + DFS_DATANODE_DATA_DIR_KEY + ": " + locationString, ioe); + } + + locations.add(location); + } + + return locations; } /** Instantiate & Start a single datanode daemon and wait for it to finish. @@ -1751,57 +1801,52 @@ public class DataNode extends Configured * no directory from this directory list can be created. * @throws IOException */ - static DataNode makeInstance(Collection dataDirs, Configuration conf, - SecureResources resources) throws IOException { + static DataNode makeInstance(Collection dataDirs, + Configuration conf, SecureResources resources) throws IOException { LocalFileSystem localFS = FileSystem.getLocal(conf); FsPermission permission = new FsPermission( conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY, DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT)); DataNodeDiskChecker dataNodeDiskChecker = new DataNodeDiskChecker(permission); - ArrayList dirs = - getDataDirsFromURIs(dataDirs, localFS, dataNodeDiskChecker); + List locations = + checkStorageLocations(dataDirs, localFS, dataNodeDiskChecker); DefaultMetricsSystem.initialize("DataNode"); - assert dirs.size() > 0 : "number of data directories should be > 0"; - return new DataNode(conf, dirs, resources); + assert locations.size() > 0 : "number of data directories should be > 0"; + return new DataNode(conf, locations, resources); } // DataNode ctor expects AbstractList instead of List or Collection... - static ArrayList getDataDirsFromURIs(Collection dataDirs, + static List checkStorageLocations( + Collection dataDirs, LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker) throws IOException { - ArrayList dirs = new ArrayList(); + ArrayList locations = new ArrayList(); StringBuilder invalidDirs = new StringBuilder(); - for (URI dirURI : dataDirs) { - if (!"file".equalsIgnoreCase(dirURI.getScheme())) { - LOG.warn("Unsupported URI schema in " + dirURI + ". Ignoring ..."); - invalidDirs.append("\"").append(dirURI).append("\" "); - continue; - } - // drop any (illegal) authority in the URI for backwards compatibility - File dir = new File(dirURI.getPath()); + for (StorageLocation location : dataDirs) { + final URI uri = location.getUri(); try { - dataNodeDiskChecker.checkDir(localFS, new Path(dir.toURI())); - dirs.add(dir); + dataNodeDiskChecker.checkDir(localFS, new Path(uri)); + locations.add(location); } catch (IOException ioe) { LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " " - + dir + " : ", ioe); - invalidDirs.append("\"").append(dirURI.getPath()).append("\" "); + + location.getFile() + " : ", ioe); + invalidDirs.append("\"").append(uri.getPath()).append("\" "); } } - if (dirs.size() == 0) { + if (locations.size() == 0) { throw new IOException("All directories in " + DFS_DATANODE_DATA_DIR_KEY + " are invalid: " + invalidDirs); } - return dirs; + return locations; } @Override public String toString() { return "DataNode{data=" + data + ", localName='" + getDisplayName() - + "', storageID='" + getStorageId() + "', xmitsInProgress=" + + "', datanodeUuid='" + storage.getDatanodeUuid() + "', xmitsInProgress=" + xmitsInProgress.get() + "}"; } @@ -1855,7 +1900,6 @@ public class DataNode extends Configured } /** - * This method is used for testing. * Examples are adding and deleting blocks directly. * The most common usage will be when the data node's storage is simulated. * @@ -1955,7 +1999,7 @@ public class DataNode extends Configured ExtendedBlock newBlock = new ExtendedBlock(oldBlock); newBlock.setGenerationStamp(recoveryId); newBlock.setNumBytes(newLength); - notifyNamenodeReceivedBlock(newBlock, ""); + notifyNamenodeReceivedBlock(newBlock, "", storageID); return storageID; } @@ -2415,6 +2459,10 @@ public class DataNode extends Configured return dnConf; } + public String getDatanodeUuid() { + return id == null ? null : id.getDatanodeUuid(); + } + boolean shouldRun() { return shouldRun; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java index f5ee53da5e7..09facf305d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java @@ -24,13 +24,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.RandomAccessFile; import java.nio.channels.FileLock; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Properties; +import java.util.*; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -50,6 +44,7 @@ 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.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.Daemon; @@ -71,8 +66,13 @@ public class DataStorage extends Storage { public final static String STORAGE_DIR_FINALIZED = "finalized"; public final static String STORAGE_DIR_TMP = "tmp"; - /** Unique storage ID. {@see DataNode#createNewStorageId(int)} for details */ - private String storageID; + /** + * Datanode UUID that this storage is currently attached to. This + * is the same as the legacy StorageID for datanodes that were + * upgraded from a pre-UUID version. For compatibility with prior + * versions of Datanodes we cannot make this field a UUID. + */ + private String datanodeUuid = null; // Flag to ensure we only initialize storage once private boolean initialized = false; @@ -84,33 +84,29 @@ public class DataStorage extends Storage { DataStorage() { super(NodeType.DATA_NODE); - storageID = ""; } public StorageInfo getBPStorage(String bpid) { return bpStorageMap.get(bpid); } - public DataStorage(StorageInfo storageInfo, String strgID) { + public DataStorage(StorageInfo storageInfo) { super(NodeType.DATA_NODE, storageInfo); - this.storageID = strgID; } - /** @return storage ID. */ - public synchronized String getStorageID() { - return storageID; + public synchronized String getDatanodeUuid() { + return datanodeUuid; } - - synchronized void setStorageID(String newStorageID) { - this.storageID = newStorageID; + + public synchronized void setDatanodeUuid(String newDatanodeUuid) { + this.datanodeUuid = newDatanodeUuid; } /** Create an ID for this storage. */ - public synchronized void createStorageID(int datanodePort) { - if (storageID != null && !storageID.isEmpty()) { - return; + public synchronized void createStorageID(StorageDirectory sd) { + if (sd.getStorageUuid() == null) { + sd.setStorageUuid(DatanodeStorage.generateUuid()); } - storageID = DataNode.createNewStorageId(datanodePort); } /** @@ -128,7 +124,8 @@ public class DataStorage extends Storage { * @throws IOException */ synchronized void recoverTransitionRead(DataNode datanode, - NamespaceInfo nsInfo, Collection dataDirs, StartupOption startOpt) + NamespaceInfo nsInfo, Collection dataDirs, + StartupOption startOpt) throws IOException { if (initialized) { // DN storage has been initialized, no need to do anything @@ -144,8 +141,8 @@ public class DataStorage extends Storage { // Format and recover. this.storageDirs = new ArrayList(dataDirs.size()); ArrayList dataDirStates = new ArrayList(dataDirs.size()); - for(Iterator it = dataDirs.iterator(); it.hasNext();) { - File dataDir = it.next(); + for(Iterator it = dataDirs.iterator(); it.hasNext();) { + File dataDir = it.next().getFile(); StorageDirectory sd = new StorageDirectory(dataDir); StorageState curState; try { @@ -162,7 +159,7 @@ public class DataStorage extends Storage { case NOT_FORMATTED: // format LOG.info("Storage directory " + dataDir + " is not formatted"); LOG.info("Formatting ..."); - format(sd, nsInfo); + format(sd, nsInfo, datanode.getDatanodeUuid()); break; default: // recovery part is common sd.doRecover(curState); @@ -191,11 +188,9 @@ public class DataStorage extends Storage { 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)); } - // make sure we have storage id set - if not - generate new one - createStorageID(datanode.getXferPort()); - // 3. Update all storages. Some of them might have just been formatted. this.writeAll(); @@ -214,14 +209,14 @@ public class DataStorage extends Storage { * @throws IOException on error */ void recoverTransitionRead(DataNode datanode, String bpID, NamespaceInfo nsInfo, - Collection dataDirs, StartupOption startOpt) throws IOException { + Collection dataDirs, StartupOption startOpt) throws IOException { // First ensure datanode level format/snapshot/rollback is completed recoverTransitionRead(datanode, nsInfo, dataDirs, startOpt); - + // Create list of storage directories for the block pool Collection bpDataDirs = new ArrayList(); - for(Iterator it = dataDirs.iterator(); it.hasNext();) { - File dnRoot = it.next(); + for(StorageLocation dir : dataDirs) { + File dnRoot = dir.getFile(); File bpRoot = BlockPoolSliceStorage.getBpRoot(bpID, new File(dnRoot, STORAGE_DIR_CURRENT)); bpDataDirs.add(bpRoot); @@ -263,19 +258,28 @@ public class DataStorage extends Storage { } } - void format(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException { + void format(StorageDirectory sd, NamespaceInfo nsInfo, + String datanodeUuid) throws IOException { sd.clearDirectory(); // create directory this.layoutVersion = HdfsConstants.LAYOUT_VERSION; this.clusterID = nsInfo.getClusterID(); this.namespaceID = nsInfo.getNamespaceID(); this.cTime = 0; - // store storageID as it currently is + this.datanodeUuid = datanodeUuid; + + if (sd.getStorageUuid() == null) { + // Assign a new Storage UUID. + sd.setStorageUuid(DatanodeStorage.generateUuid()); + } + writeProperties(sd); } /* * Set ClusterID, StorageID, StorageType, CTime into - * DataStorage VERSION file + * DataStorage VERSION file. + * Always called just before writing the properties to + * the VERSION file. */ @Override protected void setPropertiesFromFields(Properties props, @@ -285,7 +289,13 @@ public class DataStorage extends Storage { props.setProperty("clusterID", clusterID); props.setProperty("cTime", String.valueOf(cTime)); props.setProperty("layoutVersion", String.valueOf(layoutVersion)); - props.setProperty("storageID", getStorageID()); + props.setProperty("storageID", sd.getStorageUuid()); + + String datanodeUuid = getDatanodeUuid(); + if (datanodeUuid != null) { + props.setProperty("datanodeUuid", datanodeUuid); + } + // Set NamespaceID in version before federation if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) { props.setProperty("namespaceID", String.valueOf(namespaceID)); @@ -295,6 +305,7 @@ public class DataStorage extends Storage { /* * Read ClusterID, StorageID, StorageType, CTime from * DataStorage VERSION file and verify them. + * Always called just after reading the properties from the VERSION file. */ @Override protected void setFieldsFromProperties(Properties props, StorageDirectory sd) @@ -318,20 +329,36 @@ public class DataStorage extends Storage { setNamespaceID(props, sd); } + // valid storage id, storage id may be empty String ssid = props.getProperty("storageID"); if (ssid == null) { throw new InconsistentFSStateException(sd.getRoot(), "file " + STORAGE_FILE_VERSION + " is invalid."); } - String sid = getStorageID(); - if (!(sid.equals("") || ssid.equals("") || sid.equals(ssid))) { + String sid = sd.getStorageUuid(); + if (!(sid == null || sid.equals("") || + ssid.equals("") || sid.equals(ssid))) { throw new InconsistentFSStateException(sd.getRoot(), "has incompatible storage Id."); } - - if (sid.equals("")) { // update id only if it was empty - setStorageID(ssid); + + if (sid == null) { // update id only if it was null + sd.setStorageUuid(ssid); + } + + // Update the datanode UUID if present. + if (props.getProperty("datanodeUuid") != null) { + String dnUuid = props.getProperty("datanodeUuid"); + + if (getDatanodeUuid() == null) { + setDatanodeUuid(dnUuid); + } else if (getDatanodeUuid().compareTo(dnUuid) != 0) { + throw new InconsistentFSStateException(sd.getRoot(), + "Root " + sd.getRoot() + ": DatanodeUuid=" + dnUuid + + ", does not match " + getDatanodeUuid() + " from other" + + " StorageDirectory."); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 7e99c0c5d42..13b411e9ef6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -284,7 +284,7 @@ class DataXceiver extends Receiver implements Runnable { BlockSender.ClientTraceLog.info(String.format( "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," + " blockid: %s, srvID: %s, success: %b", - blk.getBlockId(), dnR.getStorageID(), (fis != null) + blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null) )); } if (fis != null) { @@ -317,7 +317,7 @@ class DataXceiver extends Receiver implements Runnable { clientName.length() > 0 && ClientTraceLog.isInfoEnabled() ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress, "%d", "HDFS_READ", clientName, "%d", - dnR.getStorageID(), block, "%d") + dnR.getDatanodeUuid(), block, "%d") : dnR + " Served block " + block + " to " + remoteAddress; @@ -447,6 +447,7 @@ class DataXceiver extends Receiver implements Runnable { String mirrorNode = null; // the name:port of next target String firstBadLink = ""; // first datanode that failed in connection setup Status mirrorInStatus = SUCCESS; + final String storageUuid; try { if (isDatanode || stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) { @@ -457,8 +458,10 @@ class DataXceiver extends Receiver implements Runnable { stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd, clientname, srcDataNode, datanode, requestedChecksum, cachingStrategy); + storageUuid = blockReceiver.getStorageUuid(); } else { - datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd); + storageUuid = datanode.data.recoverClose( + block, latestGenerationStamp, minBytesRcvd); } // @@ -590,7 +593,7 @@ class DataXceiver extends Receiver implements Runnable { // the block is finalized in the PacketResponder. if (isDatanode || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) { - datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT); + datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, storageUuid); LOG.info("Received " + block + " src: " + remoteAddress + " dest: " + localAddress + " of size " + block.getNumBytes()); } @@ -859,9 +862,11 @@ class DataXceiver extends Receiver implements Runnable { dataXceiverServer.balanceThrottler, null); // notify name node - datanode.notifyNamenodeReceivedBlock(block, delHint); + datanode.notifyNamenodeReceivedBlock( + block, delHint, blockReceiver.getStorageUuid()); - LOG.info("Moved " + block + " from " + peer.getRemoteAddressString()); + LOG.info("Moved " + block + " from " + peer.getRemoteAddressString() + + ", delHint=" + delHint); } catch (IOException ioe) { opStatus = ERROR; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java index 1258f0e4981..9ebd5740efd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.datanode; import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.LinkedList; @@ -230,10 +229,6 @@ public class DirectoryScanner implements Runnable { throw new RuntimeException(prefix + " is not a prefix of " + fullPath); } - ScanInfo(long blockId) { - this(blockId, null, null, null); - } - ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) { this.blockId = blockId; String condensedVolPath = vol == null ? null : @@ -439,8 +434,8 @@ public class DirectoryScanner implements Runnable { diffs.put(bpid, diffRecord); statsRecord.totalBlocks = blockpoolReport.length; - List bl = dataset.getFinalizedBlocks(bpid); - Block[] memReport = bl.toArray(new Block[bl.size()]); + List bl = dataset.getFinalizedBlocks(bpid); + FinalizedReplica[] memReport = bl.toArray(new FinalizedReplica[bl.size()]); Arrays.sort(memReport); // Sort based on blockId int d = 0; // index for blockpoolReport @@ -458,7 +453,8 @@ public class DirectoryScanner implements Runnable { } if (info.getBlockId() > memBlock.getBlockId()) { // Block is missing on the disk - addDifference(diffRecord, statsRecord, memBlock.getBlockId()); + addDifference(diffRecord, statsRecord, + memBlock.getBlockId(), info.getVolume()); m++; continue; } @@ -478,7 +474,9 @@ public class DirectoryScanner implements Runnable { m++; } while (m < memReport.length) { - addDifference(diffRecord, statsRecord, memReport[m++].getBlockId()); + FinalizedReplica current = memReport[m++]; + addDifference(diffRecord, statsRecord, + current.getBlockId(), current.getVolume()); } while (d < blockpoolReport.length) { statsRecord.missingMemoryBlocks++; @@ -502,10 +500,11 @@ public class DirectoryScanner implements Runnable { /** Block is not found on the disk */ private void addDifference(LinkedList diffRecord, - Stats statsRecord, long blockId) { + Stats statsRecord, long blockId, + FsVolumeSpi vol) { statsRecord.missingBlockFile++; statsRecord.missingMetaFile++; - diffRecord.add(new ScanInfo(blockId)); + diffRecord.add(new ScanInfo(blockId, null, null, vol)); } /** Is the given volume still valid in the dataset? */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java index bd0485394a3..a480bb161f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java @@ -54,4 +54,9 @@ public interface Replica { * @return the number of bytes that are visible to readers */ public long getVisibleLength(); + + /** + * Return the storageUuid of the volume that stores this replica. + */ + public String getStorageUuid(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java index 76bd1c0c6a4..27467f23ff3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java @@ -137,6 +137,14 @@ abstract public class ReplicaInfo extends Block implements Replica { void setVolume(FsVolumeSpi vol) { this.volume = vol; } + + /** + * Get the storageUuid of the volume that stores this replica. + */ + @Override + public String getStorageUuid() { + return volume.getStorageID(); + } /** * Return the parent directory path where this replica is located diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java new file mode 100644 index 00000000000..66ab18f31fb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java @@ -0,0 +1,101 @@ +/** + * 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.regex.Pattern; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.regex.Matcher; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.StorageType; +import org.apache.hadoop.hdfs.server.common.Util; + +/** + * Encapsulates the URI and storage medium that together describe a + * storage directory. + * The default storage medium is assumed to be DISK, if none is specified. + * + */ +@InterfaceAudience.Private +public class StorageLocation { + final StorageType storageType; + final File file; + + /** Regular expression that describes a storage uri with a storage type. + * e.g. [Disk]/storages/storage1/ + */ + private static final Pattern regex = Pattern.compile("^\\[(\\w*)\\](.+)$"); + + private StorageLocation(StorageType storageType, URI uri) { + this.storageType = storageType; + + if (uri.getScheme() == null || + "file".equalsIgnoreCase(uri.getScheme())) { + // drop any (illegal) authority in the URI for backwards compatibility + this.file = new File(uri.getPath()); + } else { + throw new IllegalArgumentException("Unsupported URI schema in " + uri); + } + } + + public StorageType getStorageType() { + return this.storageType; + } + + URI getUri() { + return file.toURI(); + } + + public File getFile() { + return this.file; + } + + /** + * Attempt to parse a storage uri with storage class and URI. The storage + * class component of the uri is case-insensitive. + * + * @param rawLocation Location string of the format [type]uri, where [type] is + * optional. + * @return A StorageLocation object if successfully parsed, null otherwise. + * Does not throw any exceptions. + */ + static StorageLocation parse(String rawLocation) throws IOException { + Matcher matcher = regex.matcher(rawLocation); + StorageType storageType = StorageType.DEFAULT; + String location = rawLocation; + + if (matcher.matches()) { + String classString = matcher.group(1); + location = matcher.group(2); + if (!classString.isEmpty()) { + storageType = StorageType.valueOf(classString.toUpperCase()); + } + } + + return new StorageLocation(storageType, Util.stringAsURI(location)); + } + + @Override + public String toString() { + return "[" + storageType + "]" + file.toURI(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java index 4f633973111..433000a2747 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java @@ -34,12 +34,15 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; 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.ReplicaInPipelineInterface; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory; 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; import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.util.ReflectionUtils; @@ -86,17 +89,18 @@ public interface FsDatasetSpi extends FSDatasetMBean { /** @return a list of volumes. */ public List getVolumes(); + /** @return one or more storage reports for attached volumes. */ + public StorageReport[] getStorageReports(String bpid) + throws IOException; + /** @return the volume that contains a replica of the block. */ public V getVolume(ExtendedBlock b); /** @return a volume information map (name => info). */ public Map getVolumeInfoMap(); - /** @return a list of block pools. */ - public String[] getBlockPoolList(); - /** @return a list of finalized blocks for the given block pool. */ - public List getFinalizedBlocks(String bpid); + public List getFinalizedBlocks(String bpid); /** * Check whether the in-memory block record matches the block on the disk, @@ -239,9 +243,10 @@ public interface FsDatasetSpi extends FSDatasetMBean { * @param b block * @param newGS the new generation stamp for the replica * @param expectedBlockLen the number of bytes the replica is expected to have + * @return the storage uuid of the replica. * @throws IOException */ - public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen + public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen ) throws IOException; /** @@ -262,12 +267,11 @@ public interface FsDatasetSpi extends FSDatasetMBean { public void unfinalizeBlock(ExtendedBlock b) throws IOException; /** - * Returns the block report - the full list of blocks stored under a - * block pool + * Returns one block report per volume. * @param bpid Block Pool Id - * @return - the block report - the full list of blocks stored + * @return - a map of DatanodeStorage to block report for the volume. */ - public BlockListAsLongs getBlockReport(String bpid); + public Map getBlockReports(String bpid); /** Does the dataset contain the block? */ public boolean contains(ExtendedBlock block); @@ -386,3 +390,4 @@ public interface FsDatasetSpi extends FSDatasetMBean { public HdfsBlocksMetadata getHdfsBlocksMetadata(List blocks) throws IOException; } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java index 58cfe2e8228..b14ef562541 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java @@ -20,10 +20,15 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset; import java.io.File; import java.io.IOException; +import org.apache.hadoop.hdfs.StorageType; + /** * This is an interface for the underlying volume. */ public interface FsVolumeSpi { + /** @return the StorageUuid of the volume */ + public String getStorageID(); + /** @return a list of block pools. */ public String[] getBlockPoolList(); @@ -38,4 +43,6 @@ public interface FsVolumeSpi { /** @return the directory for the finalized blocks in the block pool. */ public File getFinalizedDir(String bpid) throws IOException; + + public StorageType getStorageType(); } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java index 806921d2263..4c0e19e612e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java @@ -195,7 +195,7 @@ class FsDatasetAsyncDiskService { + " at file " + blockFile + ". Ignored."); } else { if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){ - datanode.notifyNamenodeDeletedBlock(block); + datanode.notifyNamenodeDeletedBlock(block, volume.getStorageID()); } volume.decDfsUsed(block.getBlockPoolId(), dfsBytes); LOG.info("Deleted " + block.getBlockPoolId() + " " diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index ba0d2a873cc..291926c3ccf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -37,13 +37,14 @@ 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; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.hdfs.DFSUtil; +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; @@ -52,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.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; @@ -65,6 +67,7 @@ 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; @@ -75,7 +78,9 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosing import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy; 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; import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.util.DataChecksum; @@ -107,6 +112,26 @@ class FsDatasetImpl implements FsDatasetSpi { return volumes.volumes; } + @Override // FsDatasetSpi + public StorageReport[] getStorageReports(String bpid) + throws IOException { + StorageReport[] reports; + synchronized (statsLock) { + reports = new StorageReport[volumes.volumes.size()]; + int i = 0; + for (FsVolumeImpl volume : volumes.volumes) { + reports[i++] = new StorageReport(volume.getStorageID(), + false, + volume.getCapacity(), + volume.getDfsUsed(), + volume.getAvailable(), + volume.getBlockPoolUsed(bpid)); + } + } + + return reports; + } + @Override public synchronized FsVolumeImpl getVolume(final ExtendedBlock b) { final ReplicaInfo r = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); @@ -168,10 +193,11 @@ class FsDatasetImpl implements FsDatasetSpi { final DataNode datanode; final FsVolumeList volumes; - final ReplicaMap volumeMap; final FsDatasetAsyncDiskService asyncDiskService; private final int validVolsRequired; + final ReplicaMap volumeMap; + // Used for synchronizing access to usage stats private final Object statsLock = new Object(); @@ -188,6 +214,7 @@ class FsDatasetImpl implements FsDatasetSpi { DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT); String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY); + Collection dataLocations = DataNode.getStorageLocations(conf); int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length; int volsFailed = volsConfigured - storage.getNumStorageDirs(); @@ -208,9 +235,15 @@ class FsDatasetImpl implements FsDatasetSpi { final List volArray = new ArrayList( storage.getNumStorageDirs()); for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) { - final File dir = storage.getStorageDir(idx).getCurrentDir(); - volArray.add(new FsVolumeImpl(this, storage.getStorageID(), dir, conf)); - LOG.info("Added volume - " + dir); + // TODO: getStorageTypeFromLocations() is only a temporary workaround and + // should be replaced with getting storage type from DataStorage (missing + // storage type now) directly. + Storage.StorageDirectory sd = storage.getStorageDir(idx); + final File dir = sd.getCurrentDir(); + final StorageType storageType = getStorageTypeFromLocations(dataLocations, dir); + volArray.add(new FsVolumeImpl(this, sd.getStorageUuid(), dir, conf, + storageType)); + LOG.info("Added volume - " + dir + ", StorageType: " + storageType); } volumeMap = new ReplicaMap(this); @@ -221,14 +254,24 @@ class FsDatasetImpl implements FsDatasetSpi { RoundRobinVolumeChoosingPolicy.class, VolumeChoosingPolicy.class), conf); volumes = new FsVolumeList(volArray, volsFailed, blockChooserImpl); - volumes.getVolumeMap(volumeMap); + volumes.initializeReplicaMaps(volumeMap); File[] roots = new File[storage.getNumStorageDirs()]; for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) { roots[idx] = storage.getStorageDir(idx).getCurrentDir(); } asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots); - registerMBean(storage.getStorageID()); + registerMBean(datanode.getDatanodeUuid()); + } + + private StorageType getStorageTypeFromLocations( + Collection dataLocations, File dir) { + for (StorageLocation dataLocation : dataLocations) { + if (dataLocation.getFile().equals(dir)) { + return dataLocation.getStorageType(); + } + } + return StorageType.DEFAULT; } /** @@ -308,9 +351,6 @@ class FsDatasetImpl implements FsDatasetSpi { File getBlockFile(String bpid, Block b) throws IOException { File f = validateBlockFile(bpid, b); if(f == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("b=" + b + ", volumeMap=" + volumeMap); - } throw new IOException("Block " + b + " is not valid."); } return f; @@ -654,7 +694,7 @@ class FsDatasetImpl implements FsDatasetSpi { } @Override // FsDatasetSpi - public void recoverClose(ExtendedBlock b, long newGS, + public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { LOG.info("Recover failed close " + b); // check replica's state @@ -665,6 +705,7 @@ class FsDatasetImpl implements FsDatasetSpi { if (replicaInfo.getState() == ReplicaState.RBW) { finalizeReplica(b.getBlockPoolId(), replicaInfo); } + return replicaInfo.getStorageUuid(); } /** @@ -965,51 +1006,63 @@ class FsDatasetImpl implements FsDatasetSpi { return true; } - /** - * Generates a block report from the in-memory block map. - */ - @Override // FsDatasetSpi - public BlockListAsLongs getBlockReport(String bpid) { - int size = volumeMap.size(bpid); - ArrayList finalized = new ArrayList(size); - ArrayList uc = new ArrayList(); - if (size == 0) { - return new BlockListAsLongs(finalized, uc); + @Override + public Map getBlockReports(String bpid) { + Map blockReportsMap = + new HashMap(); + + Map> finalized = + new HashMap>(); + Map> uc = + new HashMap>(); + + for (FsVolumeSpi v : volumes.volumes) { + finalized.put(v.getStorageID(), new ArrayList()); + uc.put(v.getStorageID(), new ArrayList()); } - + synchronized(this) { for (ReplicaInfo b : volumeMap.replicas(bpid)) { switch(b.getState()) { - case FINALIZED: - finalized.add(b); - break; - case RBW: - case RWR: - uc.add(b); - break; - case RUR: - ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b; - uc.add(rur.getOriginalReplica()); - break; - case TEMPORARY: - break; - default: - assert false : "Illegal ReplicaInfo state."; + case FINALIZED: + finalized.get(b.getVolume().getStorageID()).add(b); + break; + case RBW: + case RWR: + uc.get(b.getVolume().getStorageID()).add(b); + break; + case RUR: + ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b; + uc.get(rur.getVolume().getStorageID()).add(rur.getOriginalReplica()); + break; + case TEMPORARY: + break; + default: + assert false : "Illegal ReplicaInfo state."; } } - return new BlockListAsLongs(finalized, uc); } + + for (FsVolumeSpi v : volumes.volumes) { + ArrayList finalizedList = finalized.get(v.getStorageID()); + ArrayList ucList = uc.get(v.getStorageID()); + blockReportsMap.put(((FsVolumeImpl) v).toDatanodeStorage(), + new BlockListAsLongs(finalizedList, ucList)); + } + + return blockReportsMap; } /** * Get the list of finalized blocks from in-memory blockmap for a block pool. */ @Override - public synchronized List getFinalizedBlocks(String bpid) { - ArrayList finalized = new ArrayList(volumeMap.size(bpid)); + public synchronized List getFinalizedBlocks(String bpid) { + ArrayList finalized = + new ArrayList(volumeMap.size(bpid)); for (ReplicaInfo b : volumeMap.replicas(bpid)) { if(b.getState() == ReplicaState.FINALIZED) { - finalized.add(new Block(b)); + finalized.add(new FinalizedReplica((FinalizedReplica)b)); } } return finalized; @@ -1230,22 +1283,15 @@ class FsDatasetImpl implements FsDatasetSpi { /** * Register the FSDataset MBean using the name - * "hadoop:service=DataNode,name=FSDatasetState-" + * "hadoop:service=DataNode,name=FSDatasetState-" */ - void registerMBean(final String storageId) { + void registerMBean(final String datanodeUuid) { // We wrap to bypass standard mbean naming convetion. // This wraping can be removed in java 6 as it is more flexible in // package naming for mbeans and their impl. - StandardMBean bean; - String storageName; - if (storageId == null || storageId.equals("")) {// Temp fix for the uninitialized storage - storageName = "UndefinedStorageId" + DFSUtil.getRandom().nextInt(); - } else { - storageName = storageId; - } try { - bean = new StandardMBean(this,FSDatasetMBean.class); - mbeanName = MBeans.register("DataNode", "FSDatasetState-" + storageName, bean); + StandardMBean bean = new StandardMBean(this,FSDatasetMBean.class); + mbeanName = MBeans.register("DataNode", "FSDatasetState-" + datanodeUuid, bean); } catch (NotCompliantMBeanException e) { LOG.warn("Error registering FSDatasetState MBean", e); } @@ -1621,7 +1667,7 @@ class FsDatasetImpl implements FsDatasetSpi { LOG.info("Adding block pool " + bpid); volumes.addBlockPool(bpid, conf); volumeMap.initBlockPool(bpid); - volumes.getVolumeMap(bpid, volumeMap); + volumes.getAllVolumesMap(bpid, volumeMap); } @Override @@ -1631,11 +1677,6 @@ class FsDatasetImpl implements FsDatasetSpi { volumes.removeBlockPool(bpid); } - @Override - public String[] getBlockPoolList() { - return volumeMap.getBlockPoolList(); - } - /** * Class for representing the Datanode volume information */ @@ -1768,3 +1809,4 @@ class FsDatasetImpl implements FsDatasetSpi { return new RollingLogsImpl(dir, prefix); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java index 95cc74e5a5b..d2b3f587ab4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java @@ -29,9 +29,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.DF; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.datanode.DataStorage; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.util.DiskChecker.DiskErrorException; /** @@ -43,6 +45,7 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException; class FsVolumeImpl implements FsVolumeSpi { private final FsDatasetImpl dataset; private final String storageID; + private final StorageType storageType; private final Map bpSlices = new HashMap(); private final File currentDir; // /current @@ -50,7 +53,7 @@ class FsVolumeImpl implements FsVolumeSpi { private final long reserved; FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir, - Configuration conf) throws IOException { + Configuration conf, StorageType storageType) throws IOException { this.dataset = dataset; this.storageID = storageID; this.reserved = conf.getLong( @@ -59,6 +62,7 @@ class FsVolumeImpl implements FsVolumeSpi { this.currentDir = currentDir; File parent = currentDir.getParentFile(); this.usage = new DF(parent, conf); + this.storageType = storageType; } File getCurrentDir() { @@ -287,7 +291,19 @@ class FsVolumeImpl implements FsVolumeSpi { } } - String getStorageID() { + @Override + public String getStorageID() { return storageID; } + + @Override + public StorageType getStorageType() { + return storageType; + } + + DatanodeStorage toDatanodeStorage() { + return new DatanodeStorage(storageID, DatanodeStorage.State.NORMAL, storageType); + } + } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java index e9d11da5cb2..89830fcf249 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java @@ -18,10 +18,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; +import java.util.*; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; @@ -56,6 +53,7 @@ class FsVolumeList { * @param blockSize free space needed on the volume * @return next volume to store the block in. */ + // TODO should choose volume with storage type synchronized FsVolumeImpl getNextVolume(long blockSize) throws IOException { return blockChooser.chooseVolume(volumes, blockSize); } @@ -92,27 +90,32 @@ class FsVolumeList { return remaining; } - void getVolumeMap(ReplicaMap volumeMap) throws IOException { + void initializeReplicaMaps(ReplicaMap globalReplicaMap) throws IOException { for (FsVolumeImpl v : volumes) { - v.getVolumeMap(volumeMap); + v.getVolumeMap(globalReplicaMap); } } - void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException { + void getAllVolumesMap(String bpid, ReplicaMap volumeMap) throws IOException { long totalStartTime = System.currentTimeMillis(); for (FsVolumeImpl v : volumes) { - 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"); + getVolumeMap(bpid, v, volumeMap); } long totalTimeTaken = System.currentTimeMillis() - totalStartTime; FsDatasetImpl.LOG.info("Total time to add all replicas to map: " + totalTimeTaken + "ms"); } + + void getVolumeMap(String bpid, FsVolumeImpl volume, ReplicaMap volumeMap) + throws IOException { + FsDatasetImpl.LOG.info("Adding replicas to map for block pool " + bpid + + " on volume " + volume + "..."); + long startTime = System.currentTimeMillis(); + volume.getVolumeMap(bpid, volumeMap); + long timeTaken = System.currentTimeMillis() - startTime; + FsDatasetImpl.LOG.info("Time to add replicas to map for block pool " + bpid + + " on volume " + volume + ": " + timeTaken + "ms"); + } /** * Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java index aa81c727071..fe478cfaee9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java @@ -117,6 +117,13 @@ class ReplicaMap { return m.put(replicaInfo.getBlockId(), replicaInfo); } } + + /** + * Add all entries from the given replica map into the local replica map. + */ + void addAll(ReplicaMap other) { + map.putAll(other.map); + } /** * Remove the replica's meta information from the map that matches diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index af822d27569..c5ca9672719 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount; @@ -360,7 +361,7 @@ public class FSDirectory implements Closeable { * Add a block to the file. Returns a reference to the added block. */ BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block, - DatanodeDescriptor targets[]) throws IOException { + DatanodeStorageInfo[] targets) throws IOException { waitForReady(); writeLock(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index f3ab5144c7c..e223bb50391 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -147,6 +147,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ClientProtocol; @@ -170,14 +171,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; -import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException; +import org.apache.hadoop.hdfs.server.blockmanagement.*; 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; @@ -215,7 +209,8 @@ import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat; import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; -import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.util.ChunkedArrayList; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; @@ -2547,7 +2542,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, } // choose targets for the new block to be allocated. - final DatanodeDescriptor targets[] = getBlockManager().chooseTarget( + final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget( src, replication, clientNode, excludedNodes, blockSize, favoredNodes); // Part II. @@ -2675,7 +2670,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, src + ". Returning previously allocated block " + lastBlockInFile); long offset = pendingFile.computeFileSize(); onRetryBlock[0] = makeLocatedBlock(lastBlockInFile, - ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(), + ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(), offset); return iip; } else { @@ -2693,11 +2688,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats, return iip; } - LocatedBlock makeLocatedBlock(Block blk, - DatanodeInfo[] locs, + LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs, long offset) throws IOException { LocatedBlock lBlk = new LocatedBlock( - getExtendedBlock(blk), locs, offset); + getExtendedBlock(blk), locs, offset, false); getBlockManager().setBlockToken( lBlk, BlockTokenSecretManager.AccessMode.WRITE); return lBlk; @@ -2705,7 +2699,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats, /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */ LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk, - final DatanodeInfo[] existings, final Set excludes, + final DatanodeInfo[] existings, final String[] storageIDs, + final Set excludes, final int numAdditionalNodes, final String clientName ) throws IOException { //check if the feature is enabled @@ -2713,7 +2708,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, final DatanodeDescriptor clientnode; final long preferredblocksize; - final List chosen; + final List chosen; checkOperation(OperationCategory.READ); byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); readLock(); @@ -2728,23 +2723,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats, clientnode = file.getClientNode(); preferredblocksize = file.getPreferredBlockSize(); - //find datanode descriptors - chosen = new ArrayList(); - for(DatanodeInfo d : existings) { - final DatanodeDescriptor descriptor = blockManager.getDatanodeManager( - ).getDatanode(d); - if (descriptor != null) { - chosen.add(descriptor); - } - } + //find datanode storages + final DatanodeManager dm = blockManager.getDatanodeManager(); + chosen = Arrays.asList(dm.getDatanodeStorageInfos(existings, storageIDs)); } finally { readUnlock(); } // choose new datanodes. - final DatanodeInfo[] targets = blockManager.getBlockPlacementPolicy( + final DatanodeStorageInfo[] targets = blockManager.getBlockPlacementPolicy( ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true, - excludes, preferredblocksize); + // TODO: get storage type from the file + excludes, preferredblocksize, StorageType.DEFAULT); final LocatedBlock lb = new LocatedBlock(blk, targets); blockManager.setBlockToken(lb, AccessMode.COPY); return lb; @@ -2918,14 +2908,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * @throws QuotaExceededException If addition of block exceeds space quota */ BlockInfo saveAllocatedBlock(String src, INodesInPath inodes, - Block newBlock, DatanodeDescriptor targets[]) throws IOException { + Block newBlock, DatanodeStorageInfo[] targets) + throws IOException { assert hasWriteLock(); BlockInfo b = dir.addBlock(src, inodes, newBlock, targets); NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". " + getBlockPoolId() + " " + b); - for (DatanodeDescriptor dn : targets) { - dn.incBlocksScheduled(); - } + DatanodeStorageInfo.incrementBlocksScheduled(targets); return b; } @@ -3388,7 +3377,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, boolean isFileClosed(String src) throws AccessControlException, UnresolvedLinkException, StandbyException, IOException { - FSPermissionChecker pc = getPermissionChecker(); + FSPermissionChecker pc = getPermissionChecker(); checkOperation(OperationCategory.READ); readLock(); try { @@ -3685,7 +3674,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock; // setup the last block locations from the blockManager if not known if (uc.getNumExpectedLocations() == 0) { - uc.setExpectedLocations(blockManager.getNodes(lastBlock)); + uc.setExpectedLocations(blockManager.getStorages(lastBlock)); } if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) { @@ -3883,32 +3872,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats, // find the DatanodeDescriptor objects // There should be no locations in the blockManager till now because the // file is underConstruction - List targetList = + ArrayList trimmedTargets = new ArrayList(newtargets.length); + ArrayList trimmedStorages = + new ArrayList(newtargets.length); if (newtargets.length > 0) { - for (DatanodeID newtarget : newtargets) { + for (int i = 0; i < newtargets.length; ++i) { // try to get targetNode DatanodeDescriptor targetNode = - blockManager.getDatanodeManager().getDatanode(newtarget); - if (targetNode != null) - targetList.add(targetNode); - else if (LOG.isDebugEnabled()) { - LOG.debug("DatanodeDescriptor (=" + newtarget + ") not found"); + blockManager.getDatanodeManager().getDatanode(newtargets[i]); + if (targetNode != null) { + trimmedTargets.add(targetNode); + trimmedStorages.add(newtargetstorages[i]); + } else if (LOG.isDebugEnabled()) { + LOG.debug("DatanodeDescriptor (=" + newtargets[i] + ") not found"); } } } - if ((closeFile) && !targetList.isEmpty()) { + if ((closeFile) && !trimmedTargets.isEmpty()) { // the file is getting closed. Insert block locations into blockManager. // Otherwise fsck will report these blocks as MISSING, especially if the // blocksReceived from Datanodes take a long time to arrive. - for (DatanodeDescriptor targetNode : targetList) { - targetNode.addBlock(storedBlock); + for (int i = 0; i < trimmedTargets.size(); i++) { + trimmedTargets.get(i).addBlock( + trimmedStorages.get(i), storedBlock); } } + // add pipeline locations into the INodeUnderConstruction - DatanodeDescriptor[] targetArray = - new DatanodeDescriptor[targetList.size()]; - pendingFile.setLastBlock(storedBlock, targetList.toArray(targetArray)); + DatanodeStorageInfo[] trimmedStorageInfos = + blockManager.getDatanodeManager().getDatanodeStorageInfos( + trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]), + trimmedStorages.toArray(new String[trimmedStorages.size()])); + pendingFile.setLastBlock(storedBlock, trimmedStorageInfos); } if (closeFile) { @@ -4112,16 +4108,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * @throws IOException */ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, - long capacity, long dfsUsed, long remaining, long blockPoolUsed, - int xceiverCount, int xmitsInProgress, int failedVolumes) + StorageReport[] reports, int xceiverCount, int xmitsInProgress, + int failedVolumes) throws IOException { readLock(); try { final int maxTransfer = blockManager.getMaxReplicationStreams() - xmitsInProgress; DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( - nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed, - xceiverCount, maxTransfer, failedVolumes); + nodeReg, reports, blockPoolId, xceiverCount, maxTransfer, failedVolumes); return new HeartbeatResponse(cmds, createHaStatusHeartbeat()); } finally { readUnlock(); @@ -5283,11 +5278,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats, } public void processIncrementalBlockReport(final DatanodeID nodeID, - final String poolId, final ReceivedDeletedBlockInfo blockInfos[]) + final String poolId, final StorageReceivedDeletedBlocks srdb) throws IOException { writeLock(); try { - blockManager.processIncrementalBlockReport(nodeID, poolId, blockInfos); + blockManager.processIncrementalBlockReport(nodeID, poolId, srdb); } finally { writeUnlock(); } @@ -5775,9 +5770,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats, for (int i = 0; i < blocks.length; i++) { ExtendedBlock blk = blocks[i].getBlock(); DatanodeInfo[] nodes = blocks[i].getLocations(); + String[] storageIDs = blocks[i].getStorageIDs(); for (int j = 0; j < nodes.length; j++) { - DatanodeInfo dn = nodes[j]; - blockManager.findAndMarkBlockAsCorrupt(blk, dn, + blockManager.findAndMarkBlockAsCorrupt(blk, nodes[j], + storageIDs == null ? null: storageIDs[j], "client machine reported it"); } } @@ -5832,7 +5828,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * @throws IOException if any error occurs */ void updatePipeline(String clientName, ExtendedBlock oldBlock, - ExtendedBlock newBlock, DatanodeID[] newNodes) + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs) throws IOException { checkOperation(OperationCategory.WRITE); CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); @@ -5853,7 +5849,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and " + oldBlock + " has different block identifier"; updatePipelineInternal(clientName, oldBlock, newBlock, newNodes, - cacheEntry != null); + newStorageIDs, cacheEntry != null); success = true; } finally { writeUnlock(); @@ -5865,7 +5861,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats, /** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, - ExtendedBlock newBlock, DatanodeID[] newNodes, boolean logRetryCache) + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs, + boolean logRetryCache) throws IOException { assert hasWriteLock(); // check the vadility of the block and lease holder name @@ -5889,15 +5886,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats, blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); // find the DatanodeDescriptor objects - final DatanodeManager dm = getBlockManager().getDatanodeManager(); - DatanodeDescriptor[] descriptors = null; - if (newNodes.length > 0) { - descriptors = new DatanodeDescriptor[newNodes.length]; - for(int i = 0; i < newNodes.length; i++) { - descriptors[i] = dm.getDatanode(newNodes[i]); - } - } - blockinfo.setExpectedLocations(descriptors); + final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager() + .getDatanodeStorageInfos(newNodes, newStorageIDs); + blockinfo.setExpectedLocations(storages); String src = leaseManager.findPath(pendingFile); dir.persistBlocks(src, pendingFile, logRetryCache); @@ -6028,7 +6019,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * @throws IOException */ Collection listCorruptFileBlocks(String path, - String[] cookieTab) throws IOException { + String[] cookieTab) throws IOException { checkSuperuserPrivilege(); checkOperation(OperationCategory.READ); readLock(); @@ -7089,3 +7080,4 @@ public class FSNamesystem implements Namesystem, FSClusterStats, } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java index 9a8934e2e08..b2ddc4e6f54 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java @@ -26,10 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; -import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection; +import org.apache.hadoop.hdfs.server.blockmanagement.*; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.Quota.Counts; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot; @@ -215,7 +212,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc */ @Override public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock, - DatanodeDescriptor[] targets) throws IOException { + DatanodeStorageInfo[] targets) throws IOException { if (numBlocks() == 0) { throw new IOException("Failed to set last block: File is empty."); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index 1a4cc996f43..c73cd8eead0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -43,8 +43,8 @@ import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.InvalidPathException; import org.apache.hadoop.fs.Options; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; @@ -61,21 +61,21 @@ 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.CorruptFileBlocks; +import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.DatanodeID; 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; -import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; 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.DSQuotaExceededException; 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.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; @@ -91,9 +91,9 @@ import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB; 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.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.namenode.NameNode; 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; @@ -194,9 +194,9 @@ class NameNodeRpcServer implements NamenodeProtocols { NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = new NamenodeProtocolServerSideTranslatorPB(this); - BlockingService NNPbService = NamenodeProtocolService + BlockingService NNPbService = NamenodeProtocolService .newReflectiveBlockingService(namenodeProtocolXlator); - + RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator = new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this); BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService @@ -216,7 +216,7 @@ class NameNodeRpcServer implements NamenodeProtocols { new HAServiceProtocolServerSideTranslatorPB(this); BlockingService haPbService = HAServiceProtocolService .newReflectiveBlockingService(haServiceProtocolXlator); - + WritableRpcEngine.ensureInitialized(); InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf); @@ -562,7 +562,8 @@ class NameNodeRpcServer implements NamenodeProtocols { @Override // ClientProtocol public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk, - final DatanodeInfo[] existings, final DatanodeInfo[] excludes, + final DatanodeInfo[] existings, final String[] existingStorageIDs, + final DatanodeInfo[] excludes, final int numAdditionalNodes, final String clientName ) throws IOException { if (LOG.isDebugEnabled()) { @@ -583,8 +584,8 @@ class NameNodeRpcServer implements NamenodeProtocols { excludeSet.add(node); } } - return namesystem.getAdditionalDatanode(src, blk, - existings, excludeSet, numAdditionalNodes, clientName); + return namesystem.getAdditionalDatanode(src, blk, existings, + existingStorageIDs, excludeSet, numAdditionalNodes, clientName); } /** * The client needs to give up on the block. @@ -632,9 +633,9 @@ class NameNodeRpcServer implements NamenodeProtocols { @Override // ClientProtocol public void updatePipeline(String clientName, ExtendedBlock oldBlock, - ExtendedBlock newBlock, DatanodeID[] newNodes) + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs) throws IOException { - namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes); + namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes, newStorageIDs); } @Override // DatanodeProtocol @@ -955,24 +956,24 @@ class NameNodeRpcServer implements NamenodeProtocols { StorageReport[] report, int xmitsInProgress, int xceiverCount, int failedVolumes) throws IOException { verifyRequest(nodeReg); - return namesystem.handleHeartbeat(nodeReg, report[0].getCapacity(), - report[0].getDfsUsed(), report[0].getRemaining(), - report[0].getBlockPoolUsed(), xceiverCount, xmitsInProgress, - failedVolumes); + return namesystem.handleHeartbeat(nodeReg, report, + xceiverCount, xmitsInProgress, failedVolumes); } @Override // DatanodeProtocol public DatanodeCommand blockReport(DatanodeRegistration nodeReg, String poolId, StorageBlockReport[] reports) throws IOException { verifyRequest(nodeReg); - BlockListAsLongs blist = new BlockListAsLongs(reports[0].getBlocks()); if(blockStateChangeLog.isDebugEnabled()) { blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: " - + "from " + nodeReg + " " + blist.getNumberOfBlocks() - + " blocks"); + + "from " + nodeReg + ", reports.length=" + reports.length); + } + final BlockManager bm = namesystem.getBlockManager(); + for(StorageBlockReport r : reports) { + final BlockListAsLongs blocks = new BlockListAsLongs(r.getBlocks()); + bm.processReport(nodeReg, r.getStorage(), poolId, blocks); } - namesystem.getBlockManager().processReport(nodeReg, poolId, blist); if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState()) return new FinalizeCommand(poolId); return null; @@ -987,8 +988,9 @@ class NameNodeRpcServer implements NamenodeProtocols { +"from "+nodeReg+" "+receivedAndDeletedBlocks.length +" blocks."); } - namesystem.processIncrementalBlockReport( - nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks()); + for(StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) { + namesystem.processIncrementalBlockReport(nodeReg, poolId, r); + } } @Override // DatanodeProtocol @@ -1212,3 +1214,4 @@ class NameNodeRpcServer implements NamenodeProtocols { return report; } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java index c93f55e73c9..564df85468e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java @@ -31,6 +31,7 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -50,6 +51,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; @@ -1111,13 +1113,12 @@ class NamenodeJspHelper { } doc.startTag("replicas"); - for (final Iterator it = blockManager != null ? - blockManager.datanodeIterator(block) : - Collections.emptyList().iterator(); - it.hasNext();) { + for(DatanodeStorageInfo storage : (blockManager != null ? + blockManager.getStorages(block) : + Collections.emptyList())) { doc.startTag("replica"); - DatanodeDescriptor dd = it.next(); + DatanodeDescriptor dd = storage.getDatanodeDescriptor(); doc.startTag("host_name"); doc.pcdata(dd.getHostName()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java index 9c7c518ff46..d8b5eaeb76c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java @@ -53,6 +53,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Options; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -61,6 +62,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; @@ -160,7 +162,7 @@ public class NamenodeWebHdfsMethods { static DatanodeInfo chooseDatanode(final NameNode namenode, final String path, final HttpOpParam.Op op, final long openOffset, - final long blocksize, Configuration conf) throws IOException { + final long blocksize, final Configuration conf) throws IOException { final BlockManager bm = namenode.getNamesystem().getBlockManager(); if (op == PutOpParam.Op.CREATE) { @@ -168,11 +170,13 @@ public class NamenodeWebHdfsMethods { final DatanodeDescriptor clientNode = bm.getDatanodeManager( ).getDatanodeByHost(getRemoteAddress()); if (clientNode != null) { - final DatanodeDescriptor[] datanodes = bm.getBlockPlacementPolicy() + final DatanodeStorageInfo[] storages = bm.getBlockPlacementPolicy() .chooseTarget(path, 1, clientNode, - new ArrayList(), false, null, blocksize); - if (datanodes.length > 0) { - return datanodes[0]; + new ArrayList(), false, null, blocksize, + // TODO: get storage type from the file + StorageType.DEFAULT); + if (storages.length > 0) { + return storages[0].getDatanodeDescriptor(); } } } else if (op == GetOpParam.Op.OPEN diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java index 7a58c6162fe..1798d664f93 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; /**************************************************** * A BlockCommand is an instruction to a datanode @@ -46,9 +47,10 @@ public class BlockCommand extends DatanodeCommand { */ public static final long NO_ACK = Long.MAX_VALUE; - String poolId; - Block blocks[]; - DatanodeInfo targets[][]; + final String poolId; + final Block[] blocks; + final DatanodeInfo[][] targets; + final String[][] targetStorageIDs; /** * Create BlockCommand for transferring blocks to another datanode @@ -60,21 +62,26 @@ public class BlockCommand extends DatanodeCommand { this.poolId = poolId; blocks = new Block[blocktargetlist.size()]; targets = new DatanodeInfo[blocks.length][]; + targetStorageIDs = new String[blocks.length][]; + for(int i = 0; i < blocks.length; i++) { BlockTargetPair p = blocktargetlist.get(i); blocks[i] = p.block; - targets[i] = p.targets; + targets[i] = DatanodeStorageInfo.toDatanodeInfos(p.targets); + targetStorageIDs[i] = DatanodeStorageInfo.toStorageIDs(p.targets); } } - private static final DatanodeInfo[][] EMPTY_TARGET = {}; + private static final DatanodeInfo[][] EMPTY_TARGET_DATANODES = {}; + private static final String[][] EMPTY_TARGET_STORAGEIDS = {}; /** * Create BlockCommand for the given action * @param blocks blocks related to the action */ public BlockCommand(int action, String poolId, Block blocks[]) { - this(action, poolId, blocks, EMPTY_TARGET); + this(action, poolId, blocks, EMPTY_TARGET_DATANODES, + EMPTY_TARGET_STORAGEIDS); } /** @@ -82,11 +89,12 @@ public class BlockCommand extends DatanodeCommand { * @param blocks blocks related to the action */ public BlockCommand(int action, String poolId, Block[] blocks, - DatanodeInfo[][] targets) { + DatanodeInfo[][] targets, String[][] targetStorageIDs) { super(action); this.poolId = poolId; this.blocks = blocks; this.targets = targets; + this.targetStorageIDs = targetStorageIDs; } public String getBlockPoolId() { @@ -100,4 +108,8 @@ public class BlockCommand extends DatanodeCommand { public DatanodeInfo[][] getTargets() { return targets; } + + public String[][] getTargetStorageIDs() { + return targetStorageIDs; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java index 6c672b1c771..e4f34ad5fe3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hdfs.server.protocol; +import java.util.Arrays; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.Block; @@ -34,12 +36,14 @@ public class BlocksWithLocations { @InterfaceAudience.Private @InterfaceStability.Evolving public static class BlockWithLocations { - Block block; - String storageIDs[]; + final Block block; + final String[] datanodeUuids; + final String[] storageIDs; /** constructor */ - public BlockWithLocations(Block block, String[] storageIDs) { + public BlockWithLocations(Block block, String[] datanodeUuids, String[] storageIDs) { this.block = block; + this.datanodeUuids = datanodeUuids; this.storageIDs = storageIDs; } @@ -48,10 +52,30 @@ public class BlocksWithLocations { return block; } - /** get the block's locations */ + /** get the block's datanode locations */ + public String[] getDatanodeUuids() { + return datanodeUuids; + } + + /** get the block's storage locations */ public String[] getStorageIDs() { return storageIDs; } + + @Override + public String toString() { + final StringBuilder b = new StringBuilder(); + b.append(block); + if (datanodeUuids.length == 0) { + return b.append("[]").toString(); + } + + b.append(storageIDs[0]).append('@').append(datanodeUuids[0]); + for(int i = 1; i < datanodeUuids.length; i++) { + b.append(", ").append(storageIDs[i]).append("@").append(datanodeUuids[i]); + } + return b.append("]").toString(); + } } private BlockWithLocations[] blocks; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java index 7815d808d62..8ab18f4b3cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java @@ -82,7 +82,7 @@ public class DatanodeRegistration extends DatanodeID public String toString() { return getClass().getSimpleName() + "(" + getIpAddr() - + ", storageID=" + getStorageID() + + ", datanodeUuid=" + getDatanodeUuid() + ", infoPort=" + getInfoPort() + ", ipcPort=" + getIpcPort() + ", storageInfo=" + storageInfo diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java index 618708fcdef..271f71091d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hdfs.server.protocol; +import org.apache.hadoop.hdfs.StorageType; + +import java.util.UUID; + /** * Class captures information of a storage in Datanode. */ @@ -29,18 +33,21 @@ public class DatanodeStorage { private final String storageID; private final State state; + private final StorageType storageType; /** - * Create a storage with {@link State#NORMAL}. + * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}. + * * @param storageID */ public DatanodeStorage(String storageID) { - this(storageID, State.NORMAL); + this(storageID, State.NORMAL, StorageType.DEFAULT); } - public DatanodeStorage(String sid, State s) { - storageID = sid; - state = s; + public DatanodeStorage(String sid, State s, StorageType sm) { + this.storageID = sid; + this.state = s; + this.storageType = sm; } public String getStorageID() { @@ -50,4 +57,37 @@ public class DatanodeStorage { public State getState() { return state; } + + public StorageType getStorageType() { + return storageType; + } + + /** + * Generate new storage ID. The format of this string can be changed + * in the future without requiring that old storage IDs be updated. + * + * @return unique storage ID + */ + public static String generateUuid() { + return "DS-" + UUID.randomUUID(); + } + + @Override + public boolean equals(Object other){ + if (other == this) { + return true; + } + + if ((other == null) || + !(other instanceof DatanodeStorage)) { + return false; + } + DatanodeStorage otherStorage = (DatanodeStorage) other; + return otherStorage.getStorageID().compareTo(getStorageID()) == 0; + } + + @Override + public int hashCode() { + return getStorageID().hashCode(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java index d018def4e21..c805f1ea455 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java @@ -27,6 +27,8 @@ public class StorageReport { private final long dfsUsed; private final long remaining; private final long blockPoolUsed; + + public static final StorageReport[] EMPTY_ARRAY = {}; public StorageReport(String sid, boolean failed, long capacity, long dfsUsed, long remaining, long bpUsed) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java index 477db456e41..10f48465e9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java @@ -126,7 +126,7 @@ class ImageLoaderCurrent implements ImageLoader { new SimpleDateFormat("yyyy-MM-dd HH:mm"); private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23, -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39, - -40, -41, -42, -43, -44, -45, -46, -47 }; + -40, -41, -42, -43, -44, -45, -46, -47, -48, -49 }; private int imageVersion = 0; private final Map subtreeMap = new HashMap(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java index db4adc52476..441d234ce24 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java @@ -276,10 +276,11 @@ public class JsonUtil { return null; } + // TODO: Fix storageID final Map m = new TreeMap(); m.put("ipAddr", datanodeinfo.getIpAddr()); m.put("hostName", datanodeinfo.getHostName()); - m.put("storageID", datanodeinfo.getStorageID()); + m.put("storageID", datanodeinfo.getDatanodeUuid()); m.put("xferPort", datanodeinfo.getXferPort()); m.put("infoPort", datanodeinfo.getInfoPort()); m.put("infoSecurePort", datanodeinfo.getInfoSecurePort()); @@ -307,6 +308,7 @@ public class JsonUtil { infoSecurePort = 0l; // same as the default value in hdfs.proto } + // TODO: Fix storageID return new DatanodeInfo( (String)m.get("ipAddr"), (String)m.get("hostName"), @@ -384,7 +386,8 @@ public class JsonUtil { final long startOffset = (Long)m.get("startOffset"); final boolean isCorrupt = (Boolean)m.get("isCorrupt"); - final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt); + final LocatedBlock locatedblock = new LocatedBlock(b, locations, + null, null, startOffset, isCorrupt); locatedblock.setBlockToken(toBlockToken((Map)m.get("blockToken"))); return locatedblock; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto index 95fcc50ebd3..fda60857ce1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -141,6 +141,7 @@ message GetAdditionalDatanodeRequestProto { repeated DatanodeInfoProto excludes = 4; required uint32 numAdditionalNodes = 5; required string clientName = 6; + repeated string existingStorageUuids = 7; } message GetAdditionalDatanodeResponseProto { @@ -437,6 +438,7 @@ message UpdatePipelineRequestProto { required ExtendedBlockProto oldBlock = 2; required ExtendedBlockProto newBlock = 3; repeated DatanodeIDProto newNodes = 4; + repeated string storageIDs = 5; } message UpdatePipelineResponseProto { // void response diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto index 3b9b90b5d27..021a0e63c52 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto @@ -52,8 +52,9 @@ message DatanodeStorageProto { READ_ONLY = 1; } - required string storageID = 1; // Unique identifier for the storage + required string storageUuid = 1; optional StorageState state = 2 [default = NORMAL]; + optional StorageTypeProto storageType = 3 [default = DISK]; } /** @@ -103,10 +104,12 @@ message BlockCommandProto { INVALIDATE = 2; // Invalidate blocks SHUTDOWN = 3; // Shutdown the datanode } + required Action action = 1; required string blockPoolId = 2; repeated BlockProto blocks = 3; repeated DatanodeInfosProto targets = 4; + repeated StorageUuidsProto targetStorageUuids = 5; } /** @@ -172,7 +175,7 @@ message HeartbeatRequestProto { } message StorageReportProto { - required string storageID = 1; + required string storageUuid = 1; optional bool failed = 2 [ default = false ]; optional uint64 capacity = 3 [ default = 0 ]; optional uint64 dfsUsed = 4 [ default = 0 ]; @@ -250,7 +253,7 @@ message ReceivedDeletedBlockInfoProto { * List of blocks received and deleted for a storage. */ message StorageReceivedDeletedBlocksProto { - required string storageID = 1; + required string storageUuid = 1; repeated ReceivedDeletedBlockInfoProto blocks = 2; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto index c76f7edfa86..47f79bed169 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto @@ -65,7 +65,7 @@ message UpdateReplicaUnderRecoveryRequestProto { * Response returns updated block information */ message UpdateReplicaUnderRecoveryResponseProto { - required string storageID = 1; // ID of the storage that stores replica + optional string storageUuid = 1; // ID of the storage that stores replica } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto index 7a9d0adbd10..ae271f9623c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -50,7 +50,10 @@ message ExtendedBlockProto { message DatanodeIDProto { required string ipAddr = 1; // IP address required string hostName = 2; // hostname - required string storageID = 3; // unique storage id + required string datanodeUuid = 3; // UUID assigned to the Datanode. For + // upgraded clusters this is the same + // as the original StorageID of the + // Datanode. required uint32 xferPort = 4; // data streaming port required uint32 infoPort = 5; // datanode http port required uint32 ipcPort = 6; // ipc server port @@ -114,6 +117,20 @@ message FsPermissionProto { required uint32 perm = 1; // Actually a short - only 16bits used } +/** + * Types of recognized storage media. + */ +enum StorageTypeProto { + DISK = 1; + SSD = 2; +} + +/** + * A list of storage IDs. + */ +message StorageUuidsProto { + repeated string storageUuids = 1; +} /** * A LocatedBlock gives information about a block and its location. @@ -127,6 +144,8 @@ message LocatedBlockProto { // their locations are not part of this object required hadoop.common.TokenProto blockToken = 5; + repeated StorageTypeProto storageTypes = 7; + repeated string storageIDs = 8; } message DataEncryptionKeyProto { @@ -333,7 +352,8 @@ message BlockProto { */ message BlockWithLocationsProto { required BlockProto block = 1; // Block - repeated string storageIDs = 2; // Datanodes with replicas of the block + repeated string datanodeUuids = 2; // Datanodes with replicas of the block + repeated string storageUuids = 3; // Storages with replicas of the block } /** @@ -436,3 +456,4 @@ message SnapshotInfoProto { // TODO: do we need access time? } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index 4a5d394eada..9a453cc54c2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.Options.Rename; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; @@ -39,6 +40,7 @@ import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -46,6 +48,7 @@ import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.ShellBasedUnixGroupsMapping; @@ -797,7 +800,8 @@ public class DFSTestUtil { } private static DatanodeID getDatanodeID(String ipAddr) { - return new DatanodeID(ipAddr, "localhost", "", + return new DatanodeID(ipAddr, "localhost", + UUID.randomUUID().toString(), DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT, @@ -809,7 +813,8 @@ public class DFSTestUtil { } public static DatanodeID getLocalDatanodeID(int port) { - return new DatanodeID("127.0.0.1", "localhost", "", + return new DatanodeID("127.0.0.1", "localhost", + UUID.randomUUID().toString(), port, port, port, port); } @@ -831,8 +836,9 @@ public class DFSTestUtil { public static DatanodeInfo getDatanodeInfo(String ipAddr, String host, int port) { - return new DatanodeInfo(new DatanodeID(ipAddr, host, "", - port, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT, + return new DatanodeInfo(new DatanodeID(ipAddr, host, + UUID.randomUUID().toString(), port, + DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT)); } @@ -853,9 +859,43 @@ public class DFSTestUtil { rackLocation); } + public static DatanodeStorageInfo createDatanodeStorageInfo( + String storageID, String ip) { + return createDatanodeStorageInfo(storageID, ip, "defaultRack"); + } + public static DatanodeStorageInfo[] createDatanodeStorageInfos(String[] racks) { + return createDatanodeStorageInfos(racks.length, racks); + } + public static DatanodeStorageInfo[] createDatanodeStorageInfos(int n, String... racks) { + DatanodeStorageInfo[] storages = new DatanodeStorageInfo[n]; + for(int i = storages.length; i > 0; ) { + final String storageID = "s" + i; + final String ip = i + "." + i + "." + i + "." + i; + i--; + final String rack = i < racks.length? racks[i]: "defaultRack"; + storages[i] = createDatanodeStorageInfo(storageID, ip, rack); + } + return storages; + } + public static DatanodeStorageInfo createDatanodeStorageInfo( + String storageID, String ip, String rack) { + final DatanodeStorage storage = new DatanodeStorage(storageID); + final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(ip, rack, storage); + return BlockManagerTestUtil.newDatanodeStorageInfo(dn, storage); + } + public static DatanodeDescriptor[] toDatanodeDescriptor( + DatanodeStorageInfo[] storages) { + DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length]; + for(int i = 0; i < datanodes.length; i++) { + datanodes[i] = storages[i].getDatanodeDescriptor(); + } + return datanodes; + } + public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr, int port, String rackLocation) { - DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port, + DatanodeID dnId = new DatanodeID(ipAddr, "host", + UUID.randomUUID().toString(), port, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index 2c14c2d7970..6a0510c2ce3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -56,6 +56,7 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Random; import org.apache.commons.logging.Log; @@ -88,6 +89,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.hdfs.web.HftpFileSystem; @@ -125,6 +127,9 @@ public class MiniDFSCluster { public static final String DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY = DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + ".testing"; + // Changing this value may break some tests that assume it is 2. + public static final int DIRS_PER_DATANODE = 2; + static { DefaultMetricsSystem.setMiniClusterMode(true); } /** @@ -329,9 +334,10 @@ public class MiniDFSCluster { builder.nameNodePort, builder.nameNodeHttpPort); } - LOG.info("starting cluster with " + - builder.nnTopology.countNameNodes() + " namenodes."); - nameNodes = new NameNodeInfo[builder.nnTopology.countNameNodes()]; + final int numNameNodes = builder.nnTopology.countNameNodes(); + LOG.info("starting cluster: numNameNodes=" + numNameNodes + + ", numDataNodes=" + builder.numDataNodes); + nameNodes = new NameNodeInfo[numNameNodes]; initMiniDFSCluster(builder.conf, builder.numDataNodes, @@ -1149,15 +1155,16 @@ public class MiniDFSCluster { // Set up datanode address setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig); if (manageDfsDirs) { - File dir1 = getInstanceStorageDir(i, 0); - File dir2 = getInstanceStorageDir(i, 1); - dir1.mkdirs(); - dir2.mkdirs(); - if (!dir1.isDirectory() || !dir2.isDirectory()) { - throw new IOException("Mkdirs failed to create directory for DataNode " - + i + ": " + dir1 + " or " + dir2); + StringBuilder sb = new StringBuilder(); + for (int j = 0; j < DIRS_PER_DATANODE; ++j) { + File dir = getInstanceStorageDir(i, j); + dir.mkdirs(); + if (!dir.isDirectory()) { + throw new IOException("Mkdirs failed to create directory for DataNode " + dir); + } + sb.append((j > 0 ? "," : "") + fileAsURI(dir)); } - String dirs = fileAsURI(dir1) + "," + fileAsURI(dir2); + String dirs = sb.toString(); dnConf.set(DFS_DATANODE_DATA_DIR_KEY, dirs); conf.set(DFS_DATANODE_DATA_DIR_KEY, dirs); } @@ -1927,12 +1934,14 @@ public class MiniDFSCluster { // Wait for expected number of datanodes to start if (dnInfo.length != numDataNodes) { + LOG.info("dnInfo.length != numDataNodes"); return true; } // if one of the data nodes is not fully started, continue to wait for (DataNodeProperties dn : dataNodes) { if (!dn.datanode.isDatanodeFullyStarted()) { + LOG.info("!dn.datanode.isDatanodeFullyStarted()"); return true; } } @@ -1941,6 +1950,7 @@ public class MiniDFSCluster { // using (capacity == 0) as proxy. for (DatanodeInfo dn : dnInfo) { if (dn.getCapacity() == 0) { + LOG.info("dn.getCapacity() == 0"); return true; } } @@ -1948,6 +1958,7 @@ public class MiniDFSCluster { // If datanode dataset is not initialized then wait for (DataNodeProperties dn : dataNodes) { if (DataNodeTestUtils.getFSDataset(dn.datanode) == null) { + LOG.info("DataNodeTestUtils.getFSDataset(dn.datanode) == null"); return true; } } @@ -1967,12 +1978,12 @@ public class MiniDFSCluster { * @param dataNodeIndex - data node whose block report is desired - the index is same as for getDataNodes() * @return the block report for the specified data node */ - public Iterable getBlockReport(String bpid, int dataNodeIndex) { + public Map getBlockReport(String bpid, int dataNodeIndex) { if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) { throw new IndexOutOfBoundsException(); } final DataNode dn = dataNodes.get(dataNodeIndex).datanode; - return DataNodeTestUtils.getFSDataset(dn).getBlockReport(bpid); + return DataNodeTestUtils.getFSDataset(dn).getBlockReports(bpid); } @@ -1981,11 +1992,12 @@ public class MiniDFSCluster { * @return block reports from all data nodes * BlockListAsLongs is indexed in the same order as the list of datanodes returned by getDataNodes() */ - public Iterable[] getAllBlockReports(String bpid) { + public List> getAllBlockReports(String bpid) { int numDataNodes = dataNodes.size(); - Iterable[] result = new BlockListAsLongs[numDataNodes]; + final List> result + = new ArrayList>(numDataNodes); for (int i = 0; i < numDataNodes; ++i) { - result[i] = getBlockReport(bpid, i); + result.add(getBlockReport(bpid, i)); } return result; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java index 742f730d13e..8c3079c2a3a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java @@ -23,6 +23,7 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.Scanner; import java.util.concurrent.atomic.AtomicInteger; @@ -35,8 +36,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.SequenceFile; @@ -1376,11 +1379,14 @@ public class TestDFSShell { List files = new ArrayList(); List datanodes = cluster.getDataNodes(); String poolId = cluster.getNamesystem().getBlockPoolId(); - Iterable[] blocks = cluster.getAllBlockReports(poolId); - for(int i = 0; i < blocks.length; i++) { + List> blocks = cluster.getAllBlockReports(poolId); + for(int i = 0; i < blocks.size(); i++) { DataNode dn = datanodes.get(i); - for(Block b : blocks[i]) { - files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId())); + Map map = blocks.get(i); + for(Map.Entry e : map.entrySet()) { + for(Block b : e.getValue()) { + files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId())); + } } } return files; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java index 797d5ca38c9..980876cb86d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java @@ -237,7 +237,7 @@ public class TestDFSStartupVersions { * this iterations version 3-tuple * */ - @Test + @Test (timeout=300000) public void testVersions() throws Exception { UpgradeUtilities.initialize(); Configuration conf = UpgradeUtilities.initializeStorageStateConf(1, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java index 62565170bb0..dbec01c7b86 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs; import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.junit.Assume.assumeTrue; import java.io.File; import java.io.IOException; @@ -79,7 +81,8 @@ public class TestDatanodeConfig { DataNode dn = null; try { dn = DataNode.createDataNode(new String[]{}, conf); - } catch(IOException e) { + fail(); + } catch(Exception e) { // expecting exception here } if(dn != null) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java index e17cdbc6548..c666a8e8037 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java @@ -174,7 +174,7 @@ public class TestDatanodeRegistration { // register a datanode DatanodeID dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME, - "fake-storage-id", DN_XFER_PORT, DN_INFO_PORT, DN_INFO_SECURE_PORT, + "fake-datanode-id", DN_XFER_PORT, DN_INFO_PORT, DN_INFO_SECURE_PORT, DN_IPC_PORT); long nnCTime = cluster.getNamesystem().getFSImage().getStorage() .getCTime(); @@ -191,7 +191,7 @@ public class TestDatanodeRegistration { // register the same datanode again with a different storage ID dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME, - "changed-fake-storage-id", DN_XFER_PORT, DN_INFO_PORT, + "changed-fake-datanode-id", DN_XFER_PORT, DN_INFO_PORT, DN_INFO_SECURE_PORT, DN_IPC_PORT); dnReg = new DatanodeRegistration(dnId, mockStorageInfo, null, VersionInfo.getVersion()); @@ -227,7 +227,7 @@ public class TestDatanodeRegistration { DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class); doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion(); doReturn(123).when(mockDnReg).getXferPort(); - doReturn("fake-storage-id").when(mockDnReg).getStorageID(); + doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid(); doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo(); // Should succeed when software versions are the same. @@ -274,8 +274,7 @@ public class TestDatanodeRegistration { DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class); doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion(); - doReturn(123).when(mockDnReg).getXferPort(); - doReturn("fake-storage-id").when(mockDnReg).getStorageID(); + doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid(); doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo(); // Should succeed when software versions are the same and CTimes are the diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java index 6531fe7c050..81077c5fd8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java @@ -158,7 +158,7 @@ public class TestFileCorruption { ns.writeLock(); try { cluster.getNamesystem().getBlockManager().findAndMarkBlockAsCorrupt( - blk, new DatanodeInfo(dnR), "TEST"); + blk, new DatanodeInfo(dnR), "TEST", "STORAGE_ID"); } finally { ns.writeUnlock(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java index ab28ce27d6d..bda6b5146df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.net.InetSocketAddress; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.commons.logging.Log; @@ -32,10 +34,12 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.util.Time; import org.junit.Test; @@ -151,7 +155,7 @@ public class TestInjectionForSimulatedStorage { writeFile(cluster.getFileSystem(), testPath, numDataNodes); waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20); - Iterable[] blocksList = cluster.getAllBlockReports(bpid); + List> blocksList = cluster.getAllBlockReports(bpid); cluster.shutdown(); cluster = null; @@ -172,9 +176,11 @@ public class TestInjectionForSimulatedStorage { .build(); cluster.waitActive(); Set uniqueBlocks = new HashSet(); - for (int i=0; i map : blocksList) { + for(BlockListAsLongs blockList : map.values()) { + for(Block b : blockList) { + uniqueBlocks.add(new Block(b)); + } } } // Insert all the blocks in the first data node diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java index 3693e249e5b..d10a1662932 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java @@ -151,7 +151,7 @@ public class TestPeerCache { public void testAddAndRetrieve() throws Exception { PeerCache cache = new PeerCache(3, 100000); DatanodeID dnId = new DatanodeID("192.168.0.1", - "fakehostname", "fake_storage_id", + "fakehostname", "fake_datanode_id", 100, 101, 102, 103); FakePeer peer = new FakePeer(dnId, false); cache.put(dnId, peer); @@ -171,7 +171,7 @@ public class TestPeerCache { FakePeer peers[] = new FakePeer[CAPACITY]; for (int i = 0; i < CAPACITY; ++i) { dnIds[i] = new DatanodeID("192.168.0.1", - "fakehostname_" + i, "fake_storage_id", + "fakehostname_" + i, "fake_datanode_id", 100, 101, 102, 103); peers[i] = new FakePeer(dnIds[i], false); } @@ -202,7 +202,7 @@ public class TestPeerCache { FakePeer peers[] = new FakePeer[CAPACITY + 1]; for (int i = 0; i < dnIds.length; ++i) { dnIds[i] = new DatanodeID("192.168.0.1", - "fakehostname_" + i, "fake_storage_id_" + i, + "fakehostname_" + i, "fake_datanode_id_" + i, 100, 101, 102, 103); peers[i] = new FakePeer(dnIds[i], false); } @@ -233,7 +233,7 @@ public class TestPeerCache { final int CAPACITY = 3; PeerCache cache = new PeerCache(CAPACITY, 100000); DatanodeID dnId = new DatanodeID("192.168.0.1", - "fakehostname", "fake_storage_id", + "fakehostname", "fake_datanode_id", 100, 101, 102, 103); HashMultiset peers = HashMultiset.create(CAPACITY); for (int i = 0; i < CAPACITY; ++i) { @@ -258,7 +258,7 @@ public class TestPeerCache { final int CAPACITY = 3; PeerCache cache = new PeerCache(CAPACITY, 100000); DatanodeID dnId = new DatanodeID("192.168.0.1", - "fakehostname", "fake_storage_id", + "fakehostname", "fake_datanode_id", 100, 101, 102, 103); HashMultiset peers = HashMultiset.create(CAPACITY); for (int i = 0; i < CAPACITY; ++i) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java index 1a676897a16..362aac46003 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java @@ -453,12 +453,14 @@ public class UpgradeUtilities { */ public static void createDataNodeVersionFile(File[] parent, StorageInfo version, String bpid, String bpidToWrite) throws IOException { - DataStorage storage = new DataStorage(version, "doNotCare"); + DataStorage storage = new DataStorage(version); + storage.setDatanodeUuid("FixedDatanodeUuid"); File[] versionFiles = new File[parent.length]; for (int i = 0; i < parent.length; i++) { File versionFile = new File(parent[i], "VERSION"); StorageDirectory sd = new StorageDirectory(parent[i].getParentFile()); + storage.createStorageID(sd); storage.writeProperties(versionFile, sd); versionFiles[i] = versionFile; File bpDir = BlockPoolSliceStorage.getBpRoot(bpid, parent[i]); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java index b6c2f6ec29e..0064fd42b5c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java @@ -17,13 +17,16 @@ */ package org.apache.hadoop.hdfs.protocolPB; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; @@ -35,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto; +import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto; @@ -57,17 +61,9 @@ import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; -import org.apache.hadoop.hdfs.server.protocol.BlockCommand; -import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; +import org.apache.hadoop.hdfs.server.protocol.*; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; -import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; -import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; -import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; -import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; -import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; -import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; -import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.token.Token; @@ -149,12 +145,18 @@ public class TestPBHelper { void compare(DatanodeID dn, DatanodeID dn2) { assertEquals(dn.getIpAddr(), dn2.getIpAddr()); assertEquals(dn.getHostName(), dn2.getHostName()); - assertEquals(dn.getStorageID(), dn2.getStorageID()); + assertEquals(dn.getDatanodeUuid(), dn2.getDatanodeUuid()); assertEquals(dn.getXferPort(), dn2.getXferPort()); assertEquals(dn.getInfoPort(), dn2.getInfoPort()); assertEquals(dn.getIpcPort(), dn2.getIpcPort()); } + void compare(DatanodeStorage dns1, DatanodeStorage dns2) { + assertThat(dns2.getStorageID(), is(dns1.getStorageID())); + assertThat(dns2.getState(), is(dns1.getState())); + assertThat(dns2.getStorageType(), is(dns1.getStorageType())); + } + @Test public void testConvertBlock() { Block b = new Block(1, 100, 3); @@ -164,8 +166,10 @@ public class TestPBHelper { } private static BlockWithLocations getBlockWithLocations(int bid) { - return new BlockWithLocations(new Block(bid, 0, 1), new String[] { "dn1", - "dn2", "dn3" }); + final String[] datanodeUuids = {"dn1", "dn2", "dn3"}; + final String[] storageIDs = {"s1", "s2", "s3"}; + return new BlockWithLocations(new Block(bid, 0, 1), + datanodeUuids, storageIDs); } private void compare(BlockWithLocations locs1, BlockWithLocations locs2) { @@ -428,6 +432,30 @@ public class TestPBHelper { DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h3", AdminStates.NORMAL) }; + String[] storageIDs = {"s1", "s2", "s3"}; + StorageType[] media = { + StorageType.DISK, + StorageType.SSD, + StorageType.DISK + }; + LocatedBlock lb = new LocatedBlock( + new ExtendedBlock("bp12", 12345, 10, 53), + dnInfos, storageIDs, media, 5, false); + lb.setBlockToken(new Token( + "identifier".getBytes(), "password".getBytes(), new Text("kind"), + new Text("service"))); + return lb; + } + + private LocatedBlock createLocatedBlockNoStorageMedia() { + DatanodeInfo[] dnInfos = { + DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h1", + AdminStates.DECOMMISSION_INPROGRESS), + DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h2", + AdminStates.DECOMMISSIONED), + DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h3", + AdminStates.NORMAL) + }; LocatedBlock lb = new LocatedBlock( new ExtendedBlock("bp12", 12345, 10, 53), dnInfos, 5, false); lb.setBlockToken(new Token( @@ -444,6 +472,14 @@ public class TestPBHelper { compare(lb,lb2); } + @Test + public void testConvertLocatedBlockNoStorageMedia() { + LocatedBlock lb = createLocatedBlockNoStorageMedia(); + LocatedBlockProto lbProto = PBHelper.convert(lb); + LocatedBlock lb2 = PBHelper.convert(lbProto); + compare(lb,lb2); + } + @Test public void testConvertLocatedBlockList() { ArrayList lbl = new ArrayList(); @@ -487,6 +523,16 @@ public class TestPBHelper { compare(reg, reg2); assertEquals(reg.getSoftwareVersion(), reg2.getSoftwareVersion()); } + + @Test + public void TestConvertDatanodeStorage() { + DatanodeStorage dns1 = new DatanodeStorage( + "id1", DatanodeStorage.State.NORMAL, StorageType.SSD); + + DatanodeStorageProto proto = PBHelper.convert(dns1); + DatanodeStorage dns2 = PBHelper.convert(proto); + compare(dns1, dns2); + } @Test public void testConvertBlockCommand() { @@ -496,8 +542,9 @@ public class TestPBHelper { dnInfos[0][0] = DFSTestUtil.getLocalDatanodeInfo(); dnInfos[1][0] = DFSTestUtil.getLocalDatanodeInfo(); dnInfos[1][1] = DFSTestUtil.getLocalDatanodeInfo(); + String[][] storageIDs = {{"s00"}, {"s10", "s11"}}; BlockCommand bc = new BlockCommand(DatanodeProtocol.DNA_TRANSFER, "bp1", - blocks, dnInfos); + blocks, dnInfos, storageIDs); BlockCommandProto bcProto = PBHelper.convert(bc); BlockCommand bc2 = PBHelper.convert(bcProto); assertEquals(bc.getAction(), bc2.getAction()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java index 0d037fb8b48..1f5883c7a9e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java @@ -31,6 +31,7 @@ import java.util.concurrent.TimeoutException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -50,6 +51,7 @@ import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Tool; +import org.apache.log4j.Level; import org.junit.Test; /** @@ -58,7 +60,10 @@ import org.junit.Test; public class TestBalancer { private static final Log LOG = LogFactory.getLog( "org.apache.hadoop.hdfs.TestBalancer"); - + static { + ((Log4JLogger)Balancer.LOG).getLogger().setLevel(Level.ALL); + } + final static long CAPACITY = 500L; final static String RACK0 = "/rack0"; final static String RACK1 = "/rack1"; @@ -292,6 +297,16 @@ public class TestBalancer { } while (!balanced); } + String long2String(long[] array) { + if (array.length == 0) { + return ""; + } + StringBuilder b = new StringBuilder("[").append(array[0]); + for(int i = 1; i < array.length; i++) { + b.append(", ").append(array[i]); + } + return b.append("]").toString(); + } /** This test start a cluster with specified number of nodes, * and fills it to be 30% full (with a single file replicated identically * to all datanodes); @@ -308,6 +323,11 @@ public class TestBalancer { */ private void doTest(Configuration conf, long[] capacities, String[] racks, long newCapacity, String newRack, boolean useTool) throws Exception { + LOG.info("capacities = " + long2String(capacities)); + LOG.info("racks = " + Arrays.asList(racks)); + LOG.info("newCapacity= " + newCapacity); + LOG.info("newRack = " + newRack); + LOG.info("useTool = " + useTool); assertEquals(capacities.length, racks.length); int numOfDatanodes = capacities.length; cluster = new MiniDFSCluster.Builder(conf) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java index 35639fd0763..cfc9750967b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java @@ -18,15 +18,18 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; -import java.util.Iterator; import java.util.Set; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.util.Daemon; import org.junit.Assert; @@ -83,9 +86,8 @@ public class BlockManagerTestUtil { final Set rackSet = new HashSet(0); final Collection corruptNodes = getCorruptReplicas(blockManager).getNodes(b); - for (Iterator it = blockManager.blocksMap.nodeIterator(b); - it.hasNext();) { - DatanodeDescriptor cur = it.next(); + for(DatanodeStorageInfo storage : blockManager.blocksMap.getStorages(b)) { + final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { if ((corruptNodes == null ) || !corruptNodes.contains(cur)) { String rackName = cur.getNetworkLocation(); @@ -215,4 +217,52 @@ public class BlockManagerTestUtil { public static void checkHeartbeat(BlockManager bm) { bm.getDatanodeManager().getHeartbeatManager().heartbeatCheck(); } + + public static DatanodeStorageInfo updateStorage(DatanodeDescriptor dn, + DatanodeStorage s) { + return dn.updateStorage(s); + } + + public static DatanodeDescriptor getLocalDatanodeDescriptor( + boolean initializeStorage) { + DatanodeDescriptor dn = new DatanodeDescriptor(DFSTestUtil.getLocalDatanodeID()); + if (initializeStorage) { + dn.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid())); + } + return dn; + } + + public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr, + String rackLocation, boolean initializeStorage) { + return getDatanodeDescriptor(ipAddr, rackLocation, + initializeStorage? new DatanodeStorage(DatanodeStorage.generateUuid()): null); + } + + public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr, + String rackLocation, DatanodeStorage storage) { + DatanodeDescriptor dn = DFSTestUtil.getDatanodeDescriptor(ipAddr, + DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT, rackLocation); + if (storage != null) { + dn.updateStorage(storage); + } + return dn; + } + + public static DatanodeStorageInfo newDatanodeStorageInfo( + DatanodeDescriptor dn, DatanodeStorage s) { + return new DatanodeStorageInfo(dn, s); + } + + public static StorageReport[] getStorageReportsForDatanode( + DatanodeDescriptor dnd) { + ArrayList reports = new ArrayList(); + for (DatanodeStorageInfo storage : dnd.getStorageInfos()) { + StorageReport report = new StorageReport( + storage.getStorageID(), false, storage.getCapacity(), + storage.getDfsUsed(), storage.getRemaining(), + storage.getBlockPoolUsed()); + reports.add(report); + } + return reports.toArray(StorageReport.EMPTY_ARRAY); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java index 8b7111f010e..78a77c44601 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java @@ -48,7 +48,7 @@ public class TestBlockInfo { final int MAX_BLOCKS = 10; - DatanodeDescriptor dd = DFSTestUtil.getLocalDatanodeDescriptor(); + DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1"); ArrayList blockList = new ArrayList(MAX_BLOCKS); ArrayList blockInfoList = new ArrayList(); int headIndex; @@ -62,7 +62,7 @@ public class TestBlockInfo { // index of the datanode should be 0 assertEquals("Find datanode should be 0", 0, blockInfoList.get(i) - .findDatanode(dd)); + .findStorageInfo(dd)); } // list length should be equal to the number of blocks we inserted @@ -76,31 +76,31 @@ public class TestBlockInfo { } assertEquals("There should be MAX_BLOCK blockInfo's", MAX_BLOCKS, len); - headIndex = dd.getHead().findDatanode(dd); + headIndex = dd.getBlockListHeadForTesting().findStorageInfo(dd); LOG.info("Moving each block to the head of the list..."); for (int i = 0; i < MAX_BLOCKS; i++) { - curIndex = blockInfoList.get(i).findDatanode(dd); + curIndex = blockInfoList.get(i).findStorageInfo(dd); headIndex = dd.moveBlockToHead(blockInfoList.get(i), curIndex, headIndex); // the moved element must be at the head of the list assertEquals("Block should be at the head of the list now.", - blockInfoList.get(i), dd.getHead()); + blockInfoList.get(i), dd.getBlockListHeadForTesting()); } // move head of the list to the head - this should not change the list LOG.info("Moving head to the head..."); - BlockInfo temp = dd.getHead(); + BlockInfo temp = dd.getBlockListHeadForTesting(); curIndex = 0; headIndex = 0; dd.moveBlockToHead(temp, curIndex, headIndex); assertEquals( "Moving head to the head of the list shopuld not change the list", - temp, dd.getHead()); + temp, dd.getBlockListHeadForTesting()); // check all elements of the list against the original blockInfoList LOG.info("Checking elements of the list..."); - temp = dd.getHead(); + temp = dd.getBlockListHeadForTesting(); assertNotNull("Head should not be null", temp); int c = MAX_BLOCKS - 1; while (temp != null) { @@ -110,15 +110,15 @@ public class TestBlockInfo { } LOG.info("Moving random blocks to the head of the list..."); - headIndex = dd.getHead().findDatanode(dd); + headIndex = dd.getBlockListHeadForTesting().findStorageInfo(dd); Random rand = new Random(); for (int i = 0; i < MAX_BLOCKS; i++) { int j = rand.nextInt(MAX_BLOCKS); - curIndex = blockInfoList.get(j).findDatanode(dd); + curIndex = blockInfoList.get(j).findStorageInfo(dd); headIndex = dd.moveBlockToHead(blockInfoList.get(j), curIndex, headIndex); // the moved element must be at the head of the list assertEquals("Block should be at the head of the list now.", - blockInfoList.get(j), dd.getHead()); + blockInfoList.get(j), dd.getBlockListHeadForTesting()); } } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java index cafc8227147..703d344347a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java @@ -31,18 +31,19 @@ import org.junit.Test; public class TestBlockInfoUnderConstruction { @Test public void testInitializeBlockRecovery() throws Exception { - DatanodeDescriptor dd1 = DFSTestUtil.getDatanodeDescriptor("10.10.1.1", - "default"); - DatanodeDescriptor dd2 = DFSTestUtil.getDatanodeDescriptor("10.10.1.2", - "default"); - DatanodeDescriptor dd3 = DFSTestUtil.getDatanodeDescriptor("10.10.1.3", - "default"); + DatanodeStorageInfo s1 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.1", "s1"); + DatanodeDescriptor dd1 = s1.getDatanodeDescriptor(); + DatanodeStorageInfo s2 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.2", "s2"); + DatanodeDescriptor dd2 = s2.getDatanodeDescriptor(); + DatanodeStorageInfo s3 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.3", "s3"); + DatanodeDescriptor dd3 = s3.getDatanodeDescriptor(); + dd1.isAlive = dd2.isAlive = dd3.isAlive = true; BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, BlockUCState.UNDER_CONSTRUCTION, - new DatanodeDescriptor[] {dd1, dd2, dd3}); + new DatanodeStorageInfo[] {s1, s2, s3}); // Recovery attempt #1. long currentTime = System.currentTimeMillis(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java index e88ec92e39b..4f69c5d499f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java @@ -22,9 +22,14 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.LinkedList; import java.util.List; import java.util.Map.Entry; @@ -39,11 +44,11 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.net.NetworkTopology; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import static org.mockito.Mockito.*; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; @@ -51,6 +56,7 @@ import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Lists; public class TestBlockManager { + private DatanodeStorageInfo[] storages; private List nodes; private List rackA; private List rackB; @@ -79,26 +85,29 @@ public class TestBlockManager { fsn = Mockito.mock(FSNamesystem.class); Mockito.doReturn(true).when(fsn).hasWriteLock(); bm = new BlockManager(fsn, fsn, conf); - nodes = ImmutableList.of( - DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackB"), - DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackB"), - DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackB") - ); + final String[] racks = { + "/rackA", + "/rackA", + "/rackA", + "/rackB", + "/rackB", + "/rackB"}; + storages = DFSTestUtil.createDatanodeStorageInfos(racks); + nodes = Arrays.asList(DFSTestUtil.toDatanodeDescriptor(storages)); rackA = nodes.subList(0, 3); rackB = nodes.subList(3, 6); } - + private void addNodes(Iterable nodesToAdd) { NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology(); // construct network topology for (DatanodeDescriptor dn : nodesToAdd) { cluster.add(dn); + dn.getStorageInfos()[0].setUtilizationForTesting( + 2 * HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, + 2 * HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L); dn.updateHeartbeat( - 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, - 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); + BlockManagerTestUtil.getStorageReportsForDatanode(dn), 0, 0); bm.getDatanodeManager().checkIfClusterIsNowMultiRack(dn); } } @@ -123,17 +132,18 @@ public class TestBlockManager { } private void doBasicTest(int testIndex) { - List origNodes = getNodes(0, 1); - BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes); + List origStorages = getStorages(0, 1); + List origNodes = getNodes(origStorages); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); - DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo); + DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo); assertEquals(2, pipeline.length); assertTrue("Source of replication should be one of the nodes the block " + "was on. Was: " + pipeline[0], - origNodes.contains(pipeline[0])); + origStorages.contains(pipeline[0])); assertTrue("Destination of replication should be on the other rack. " + "Was: " + pipeline[1], - rackB.contains(pipeline[1])); + rackB.contains(pipeline[1].getDatanodeDescriptor())); } @@ -154,21 +164,22 @@ public class TestBlockManager { private void doTestTwoOfThreeNodesDecommissioned(int testIndex) throws Exception { // Block originally on A1, A2, B1 - List origNodes = getNodes(0, 1, 3); + List origStorages = getStorages(0, 1, 3); + List origNodes = getNodes(origStorages); BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); // Decommission two of the nodes (A1, A2) List decomNodes = startDecommission(0, 1); - DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo); + DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo); assertTrue("Source of replication should be one of the nodes the block " + "was on. Was: " + pipeline[0], - origNodes.contains(pipeline[0])); + origStorages.contains(pipeline[0])); assertEquals("Should have three targets", 3, pipeline.length); boolean foundOneOnRackA = false; for (int i = 1; i < pipeline.length; i++) { - DatanodeDescriptor target = pipeline[i]; + DatanodeDescriptor target = pipeline[i].getDatanodeDescriptor(); if (rackA.contains(target)) { foundOneOnRackA = true; } @@ -197,22 +208,23 @@ public class TestBlockManager { private void doTestAllNodesHoldingReplicasDecommissioned(int testIndex) throws Exception { // Block originally on A1, A2, B1 - List origNodes = getNodes(0, 1, 3); + List origStorages = getStorages(0, 1, 3); + List origNodes = getNodes(origStorages); BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); // Decommission all of the nodes List decomNodes = startDecommission(0, 1, 3); - DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo); + DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo); assertTrue("Source of replication should be one of the nodes the block " + "was on. Was: " + pipeline[0], - origNodes.contains(pipeline[0])); + origStorages.contains(pipeline[0])); assertEquals("Should have three targets", 4, pipeline.length); boolean foundOneOnRackA = false; boolean foundOneOnRackB = false; for (int i = 1; i < pipeline.length; i++) { - DatanodeDescriptor target = pipeline[i]; + DatanodeDescriptor target = pipeline[i].getDatanodeDescriptor(); if (rackA.contains(target)) { foundOneOnRackA = true; } else if (rackB.contains(target)) { @@ -249,21 +261,22 @@ public class TestBlockManager { private void doTestOneOfTwoRacksDecommissioned(int testIndex) throws Exception { // Block originally on A1, A2, B1 - List origNodes = getNodes(0, 1, 3); + List origStorages = getStorages(0, 1, 3); + List origNodes = getNodes(origStorages); BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); // Decommission all of the nodes in rack A List decomNodes = startDecommission(0, 1, 2); - DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo); + DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo); assertTrue("Source of replication should be one of the nodes the block " + "was on. Was: " + pipeline[0], - origNodes.contains(pipeline[0])); + origStorages.contains(pipeline[0])); assertEquals("Should have three targets", 3, pipeline.length); boolean foundOneOnRackB = false; for (int i = 1; i < pipeline.length; i++) { - DatanodeDescriptor target = pipeline[i]; + DatanodeDescriptor target = pipeline[i].getDatanodeDescriptor(); if (rackB.contains(target)) { foundOneOnRackB = true; } @@ -282,11 +295,12 @@ public class TestBlockManager { // the third off-rack replica. DatanodeDescriptor rackCNode = DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/rackC"); + rackCNode.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid())); addNodes(ImmutableList.of(rackCNode)); try { - DatanodeDescriptor[] pipeline2 = scheduleSingleReplication(blockInfo); + DatanodeStorageInfo[] pipeline2 = scheduleSingleReplication(blockInfo); assertEquals(2, pipeline2.length); - assertEquals(rackCNode, pipeline2[1]); + assertEquals(rackCNode, pipeline2[1].getDatanodeDescriptor()); } finally { removeNode(rackCNode); } @@ -307,30 +321,30 @@ public class TestBlockManager { private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) { // Originally on only nodes in rack A. List origNodes = rackA; - BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes); - DatanodeDescriptor pipeline[] = scheduleSingleReplication(blockInfo); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); + DatanodeStorageInfo pipeline[] = scheduleSingleReplication(blockInfo); assertEquals(2, pipeline.length); // single new copy assertTrue("Source of replication should be one of the nodes the block " + "was on. Was: " + pipeline[0], - origNodes.contains(pipeline[0])); + origNodes.contains(pipeline[0].getDatanodeDescriptor())); assertTrue("Destination of replication should be on the other rack. " + "Was: " + pipeline[1], - rackB.contains(pipeline[1])); + rackB.contains(pipeline[1].getDatanodeDescriptor())); } @Test public void testBlocksAreNotUnderreplicatedInSingleRack() throws Exception { List nodes = ImmutableList.of( - DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackA"), - DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackA") + BlockManagerTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA", true), + BlockManagerTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA", true), + BlockManagerTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA", true), + BlockManagerTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackA", true), + BlockManagerTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackA", true), + BlockManagerTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackA", true) ); addNodes(nodes); - List origNodes = nodes.subList(0, 3);; + List origNodes = nodes.subList(0, 3); for (int i = 0; i < NUM_TEST_ITERS; i++) { doTestSingleRackClusterIsSufficientlyReplicated(i, origNodes); } @@ -340,7 +354,7 @@ public class TestBlockManager { List origNodes) throws Exception { assertEquals(0, bm.numOfUnderReplicatedBlocks()); - addBlockOnNodes((long)testIndex, origNodes); + addBlockOnNodes(testIndex, origNodes); bm.processMisReplicatedBlocks(); assertEquals(0, bm.numOfUnderReplicatedBlocks()); } @@ -351,9 +365,11 @@ public class TestBlockManager { * pipeline. */ private void fulfillPipeline(BlockInfo blockInfo, - DatanodeDescriptor[] pipeline) throws IOException { + DatanodeStorageInfo[] pipeline) throws IOException { for (int i = 1; i < pipeline.length; i++) { - bm.addBlock(pipeline[i], blockInfo, null); + DatanodeStorageInfo storage = pipeline[i]; + bm.addBlock(storage.getDatanodeDescriptor(), storage.getStorageID(), blockInfo, null); + blockInfo.addStorage(storage); } } @@ -362,7 +378,9 @@ public class TestBlockManager { BlockInfo blockInfo = new BlockInfo(block, 3); for (DatanodeDescriptor dn : nodes) { - blockInfo.addNode(dn); + for (DatanodeStorageInfo storage : dn.getStorageInfos()) { + blockInfo.addStorage(storage); + } } return blockInfo; } @@ -374,6 +392,22 @@ public class TestBlockManager { } return ret; } + + private List getNodes(List storages) { + List ret = Lists.newArrayList(); + for (DatanodeStorageInfo s : storages) { + ret.add(s.getDatanodeDescriptor()); + } + return ret; + } + + private List getStorages(int ... indexes) { + List ret = Lists.newArrayList(); + for (int idx : indexes) { + ret.add(storages[idx]); + } + return ret; + } private List startDecommission(int ... indexes) { List nodes = getNodes(indexes); @@ -392,7 +426,7 @@ public class TestBlockManager { return blockInfo; } - private DatanodeDescriptor[] scheduleSingleReplication(Block block) { + private DatanodeStorageInfo[] scheduleSingleReplication(Block block) { // list for priority 1 List list_p1 = new ArrayList(); list_p1.add(block); @@ -410,27 +444,29 @@ public class TestBlockManager { assertTrue("replication is pending after work is computed", bm.pendingReplications.getNumReplicas(block) > 0); - LinkedListMultimap repls = getAllPendingReplications(); + LinkedListMultimap repls = getAllPendingReplications(); assertEquals(1, repls.size()); - Entry repl = + Entry repl = repls.entries().iterator().next(); - DatanodeDescriptor[] targets = repl.getValue().targets; + DatanodeStorageInfo[] targets = repl.getValue().targets; - DatanodeDescriptor[] pipeline = new DatanodeDescriptor[1 + targets.length]; + DatanodeStorageInfo[] pipeline = new DatanodeStorageInfo[1 + targets.length]; pipeline[0] = repl.getKey(); System.arraycopy(targets, 0, pipeline, 1, targets.length); return pipeline; } - private LinkedListMultimap getAllPendingReplications() { - LinkedListMultimap repls = + private LinkedListMultimap getAllPendingReplications() { + LinkedListMultimap repls = LinkedListMultimap.create(); for (DatanodeDescriptor dn : nodes) { List thisRepls = dn.getReplicationCommand(10); if (thisRepls != null) { - repls.putAll(dn, thisRepls); + for(DatanodeStorageInfo storage : dn.getStorageInfos()) { + repls.putAll(storage, thisRepls); + } } } return repls; @@ -453,7 +489,7 @@ public class TestBlockManager { addBlockOnNodes(blockId,origNodes.subList(0,1)); List cntNodes = new LinkedList(); - List liveNodes = new LinkedList(); + List liveNodes = new LinkedList(); assertNotNull("Chooses source node for a highest-priority replication" + " even if all available source nodes have reached their replication" @@ -476,7 +512,7 @@ public class TestBlockManager { UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)); // Increase the replication count to test replication count > hard limit - DatanodeDescriptor targets[] = { origNodes.get(1) }; + DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] }; origNodes.get(0).addBlockToBeReplicated(aBlock, targets); assertNull("Does not choose a source node for a highest-priority" @@ -492,7 +528,11 @@ public class TestBlockManager { @Test public void testSafeModeIBR() throws Exception { DatanodeDescriptor node = spy(nodes.get(0)); - node.setStorageID("dummy-storage"); + DatanodeStorageInfo ds = node.getStorageInfos()[0]; + + // TODO: Needs to be fixed. DatanodeUuid is not storageID. + node.setDatanodeUuidForTesting(ds.getStorageID()); + node.isAlive = true; DatanodeRegistration nodeReg = @@ -505,35 +545,40 @@ public class TestBlockManager { bm.getDatanodeManager().registerDatanode(nodeReg); bm.getDatanodeManager().addDatanode(node); // swap in spy assertEquals(node, bm.getDatanodeManager().getDatanode(node)); - assertTrue(node.isFirstBlockReport()); + assertEquals(0, ds.getBlockReportCount()); // send block report, should be processed reset(node); - bm.processReport(node, "pool", new BlockListAsLongs(null, null)); - verify(node).receivedBlockReport(); - assertFalse(node.isFirstBlockReport()); + + bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool", + new BlockListAsLongs(null, null)); + assertEquals(1, ds.getBlockReportCount()); // send block report again, should NOT be processed reset(node); - bm.processReport(node, "pool", new BlockListAsLongs(null, null)); - verify(node, never()).receivedBlockReport(); - assertFalse(node.isFirstBlockReport()); + bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool", + new BlockListAsLongs(null, null)); + assertEquals(1, ds.getBlockReportCount()); // re-register as if node restarted, should update existing node bm.getDatanodeManager().removeDatanode(node); reset(node); bm.getDatanodeManager().registerDatanode(nodeReg); verify(node).updateRegInfo(nodeReg); - assertTrue(node.isFirstBlockReport()); // ready for report again + assertEquals(0, ds.getBlockReportCount()); // ready for report again // send block report, should be processed after restart reset(node); - bm.processReport(node, "pool", new BlockListAsLongs(null, null)); - verify(node).receivedBlockReport(); - assertFalse(node.isFirstBlockReport()); + bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool", + new BlockListAsLongs(null, null)); + assertEquals(1, ds.getBlockReportCount()); } @Test public void testSafeModeIBRAfterIncremental() throws Exception { DatanodeDescriptor node = spy(nodes.get(0)); - node.setStorageID("dummy-storage"); + DatanodeStorageInfo ds = node.getStorageInfos()[0]; + + // TODO: Needs to be fixed. DatanodeUuid is not storageID. + node.setDatanodeUuidForTesting(ds.getStorageID()); + node.isAlive = true; DatanodeRegistration nodeReg = @@ -546,12 +591,13 @@ public class TestBlockManager { bm.getDatanodeManager().registerDatanode(nodeReg); bm.getDatanodeManager().addDatanode(node); // swap in spy assertEquals(node, bm.getDatanodeManager().getDatanode(node)); - assertTrue(node.isFirstBlockReport()); + assertEquals(0, ds.getBlockReportCount()); // send block report while pretending to already have blocks reset(node); doReturn(1).when(node).numBlocks(); - bm.processReport(node, "pool", new BlockListAsLongs(null, null)); - verify(node).receivedBlockReport(); - assertFalse(node.isFirstBlockReport()); + bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool", + new BlockListAsLongs(null, null)); + assertEquals(1, ds.getBlockReportCount()); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java index bd030fb94dd..12674eb318a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java @@ -55,21 +55,24 @@ public class TestDatanodeDescriptor { @Test public void testBlocksCounter() throws Exception { - DatanodeDescriptor dd = DFSTestUtil.getLocalDatanodeDescriptor(); + DatanodeDescriptor dd = BlockManagerTestUtil.getLocalDatanodeDescriptor(true); assertEquals(0, dd.numBlocks()); BlockInfo blk = new BlockInfo(new Block(1L), 1); BlockInfo blk1 = new BlockInfo(new Block(2L), 2); + DatanodeStorageInfo[] storages = dd.getStorageInfos(); + assertTrue(storages.length > 0); + final String storageID = storages[0].getStorageID(); // add first block - assertTrue(dd.addBlock(blk)); + assertTrue(dd.addBlock(storageID, blk)); assertEquals(1, dd.numBlocks()); // remove a non-existent block assertFalse(dd.removeBlock(blk1)); assertEquals(1, dd.numBlocks()); // add an existent block - assertFalse(dd.addBlock(blk)); + assertFalse(dd.addBlock(storageID, blk)); assertEquals(1, dd.numBlocks()); // add second block - assertTrue(dd.addBlock(blk1)); + assertTrue(dd.addBlock(storageID, blk1)); assertEquals(2, dd.numBlocks()); // remove first block assertTrue(dd.removeBlock(blk)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java index 75034932acd..84d0d6b5800 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java @@ -76,7 +76,7 @@ public class TestDatanodeManager { it.next(); } DatanodeRegistration toRemove = it.next().getValue(); - Log.info("Removing node " + toRemove.getStorageID() + " ip " + + Log.info("Removing node " + toRemove.getDatanodeUuid() + " ip " + toRemove.getXferAddr() + " version : " + toRemove.getSoftwareVersion()); //Remove that random node @@ -90,7 +90,7 @@ public class TestDatanodeManager { String storageID = "someStorageID" + rng.nextInt(5000); DatanodeRegistration dr = Mockito.mock(DatanodeRegistration.class); - Mockito.when(dr.getStorageID()).thenReturn(storageID); + Mockito.when(dr.getDatanodeUuid()).thenReturn(storageID); //If this storageID had already been registered before if(sIdToDnReg.containsKey(storageID)) { @@ -110,7 +110,7 @@ public class TestDatanodeManager { Mockito.when(dr.getSoftwareVersion()).thenReturn( "version" + rng.nextInt(5)); - Log.info("Registering node storageID: " + dr.getStorageID() + + Log.info("Registering node storageID: " + dr.getDatanodeUuid() + ", version: " + dr.getSoftwareVersion() + ", IP address: " + dr.getXferAddr()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java index bbb83070a30..4b27aa68b3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import static org.junit.Assert.assertEquals; import java.util.ArrayList; -import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -38,6 +37,7 @@ 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.DatanodeStorage; import org.junit.Test; /** @@ -63,6 +63,8 @@ public class TestHeartbeatHandling { final DatanodeRegistration nodeReg = DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId); final DatanodeDescriptor dd = NameNodeAdapter.getDatanode(namesystem, nodeReg); + final String storageID = DatanodeStorage.generateUuid(); + dd.updateStorage(new DatanodeStorage(storageID)); final int REMAINING_BLOCKS = 1; final int MAX_REPLICATE_LIMIT = @@ -70,7 +72,7 @@ public class TestHeartbeatHandling { final int MAX_INVALIDATE_LIMIT = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT; final int MAX_INVALIDATE_BLOCKS = 2*MAX_INVALIDATE_LIMIT+REMAINING_BLOCKS; final int MAX_REPLICATE_BLOCKS = 2*MAX_REPLICATE_LIMIT+REMAINING_BLOCKS; - final DatanodeDescriptor[] ONE_TARGET = new DatanodeDescriptor[1]; + final DatanodeStorageInfo[] ONE_TARGET = {dd.getStorageInfo(storageID)}; try { namesystem.writeLock(); @@ -144,12 +146,15 @@ public class TestHeartbeatHandling { final DatanodeRegistration nodeReg1 = DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId); final DatanodeDescriptor dd1 = NameNodeAdapter.getDatanode(namesystem, nodeReg1); + dd1.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid())); final DatanodeRegistration nodeReg2 = DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(1), poolId); final DatanodeDescriptor dd2 = NameNodeAdapter.getDatanode(namesystem, nodeReg2); + dd2.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid())); final DatanodeRegistration nodeReg3 = DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(2), poolId); final DatanodeDescriptor dd3 = NameNodeAdapter.getDatanode(namesystem, nodeReg3); + dd3.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid())); try { namesystem.writeLock(); @@ -162,10 +167,13 @@ public class TestHeartbeatHandling { dd1.setLastUpdate(System.currentTimeMillis()); dd2.setLastUpdate(System.currentTimeMillis()); dd3.setLastUpdate(System.currentTimeMillis()); + final DatanodeStorageInfo[] storages = { + dd1.getStorageInfos()[0], + dd2.getStorageInfos()[0], + dd3.getStorageInfos()[0]}; BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, - BlockUCState.UNDER_RECOVERY, - new DatanodeDescriptor[] {dd1, dd2, dd3}); + BlockUCState.UNDER_RECOVERY, storages); dd1.addBlockToBeRecovered(blockInfo); DatanodeCommand[] cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); @@ -187,8 +195,7 @@ public class TestHeartbeatHandling { dd3.setLastUpdate(System.currentTimeMillis()); blockInfo = new BlockInfoUnderConstruction( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, - BlockUCState.UNDER_RECOVERY, - new DatanodeDescriptor[] {dd1, dd2, dd3}); + BlockUCState.UNDER_RECOVERY, storages); dd1.addBlockToBeRecovered(blockInfo); cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); assertEquals(1, cmds.length); @@ -209,8 +216,7 @@ public class TestHeartbeatHandling { dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000); blockInfo = new BlockInfoUnderConstruction( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, - BlockUCState.UNDER_RECOVERY, - new DatanodeDescriptor[] {dd1, dd2, dd3}); + BlockUCState.UNDER_RECOVERY, storages); dd1.addBlockToBeRecovered(blockInfo); cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); assertEquals(1, cmds.length); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java index 86994bed3ac..f7973819ecb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import static org.junit.Assert.assertTrue; import java.util.Collection; -import java.util.Iterator; import java.util.concurrent.TimeoutException; import org.apache.hadoop.conf.Configuration; @@ -98,12 +97,10 @@ public class TestNodeCount { } // find out a non-excess node - final Iterator iter = bm.blocksMap - .nodeIterator(block.getLocalBlock()); DatanodeDescriptor nonExcessDN = null; - while (iter.hasNext()) { - DatanodeDescriptor dn = iter.next(); - Collection blocks = bm.excessReplicateMap.get(dn.getStorageID()); + for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) { + final DatanodeDescriptor dn = storage.getDatanodeDescriptor(); + Collection blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid()); if (blocks == null || !blocks.contains(block.getLocalBlock()) ) { nonExcessDN = dn; break; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java index 972a785f5d6..47f8730268b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java @@ -103,7 +103,10 @@ public class TestOverReplicatedBlocks { String corruptMachineName = corruptDataNode.getXferAddr(); for (DatanodeDescriptor datanode : hm.getDatanodes()) { if (!corruptMachineName.equals(datanode.getXferAddr())) { - datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0, 0); + datanode.getStorageInfos()[0].setUtilizationForTesting(100L, 100L, 0, 100L); + datanode.updateHeartbeat( + BlockManagerTestUtil.getStorageReportsForDatanode(datanode), + 0, 0); } } @@ -155,7 +158,7 @@ public class TestOverReplicatedBlocks { DataNode lastDN = cluster.getDataNodes().get(3); DatanodeRegistration dnReg = DataNodeTestUtils.getDNRegistrationForBP( lastDN, namesystem.getBlockPoolId()); - String lastDNid = dnReg.getStorageID(); + String lastDNid = dnReg.getDatanodeUuid(); final Path fileName = new Path("/foo2"); DFSTestUtil.createFile(fs, fileName, SMALL_FILE_LENGTH, (short)4, 0L); @@ -220,3 +223,4 @@ public class TestOverReplicatedBlocks { } } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java index 3c7ad8ca021..dbff77402ff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java @@ -43,8 +43,8 @@ public class TestPendingDataNodeMessages { @Test public void testQueues() { DatanodeDescriptor fakeDN = DFSTestUtil.getLocalDatanodeDescriptor(); - msgs.enqueueReportedBlock(fakeDN, block1Gs1, ReplicaState.FINALIZED); - msgs.enqueueReportedBlock(fakeDN, block1Gs2, ReplicaState.FINALIZED); + msgs.enqueueReportedBlock(fakeDN, "STORAGE_ID", block1Gs1, ReplicaState.FINALIZED); + msgs.enqueueReportedBlock(fakeDN, "STORAGE_ID", block1Gs2, ReplicaState.FINALIZED); assertEquals(2, msgs.count()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java index 0c1fee98646..c63badc9eb7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java @@ -43,8 +43,6 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStat import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.junit.Test; -import com.google.common.base.Preconditions; - /** * This class tests the internals of PendingReplicationBlocks.java, * as well as how PendingReplicationBlocks acts in BlockManager @@ -54,22 +52,7 @@ public class TestPendingReplication { private static final int DFS_REPLICATION_INTERVAL = 1; // Number of datanodes in the cluster private static final int DATANODE_COUNT = 5; - - private DatanodeDescriptor genDatanodeId(int seed) { - seed = seed % 256; - String ip = seed + "." + seed + "." + seed + "." + seed; - return DFSTestUtil.getDatanodeDescriptor(ip, null); - } - private DatanodeDescriptor[] genDatanodes(int number) { - Preconditions.checkArgument(number >= 0); - DatanodeDescriptor[] nodes = new DatanodeDescriptor[number]; - for (int i = 0; i < number; i++) { - nodes[i] = genDatanodeId(i); - } - return nodes; - } - @Test public void testPendingReplication() { PendingReplicationBlocks pendingReplications; @@ -79,9 +62,13 @@ public class TestPendingReplication { // // Add 10 blocks to pendingReplications. // - for (int i = 0; i < 10; i++) { + DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10); + for (int i = 0; i < storages.length; i++) { Block block = new Block(i, i, 0); - pendingReplications.increment(block, genDatanodes(i)); + DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i]; + System.arraycopy(storages, 0, targets, 0, i); + pendingReplications.increment(block, + DatanodeStorageInfo.toDatanodeDescriptors(targets)); } assertEquals("Size of pendingReplications ", 10, pendingReplications.size()); @@ -91,16 +78,18 @@ public class TestPendingReplication { // remove one item and reinsert it // Block blk = new Block(8, 8, 0); - pendingReplications.decrement(blk, genDatanodeId(7)); // removes one replica + pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica assertEquals("pendingReplications.getNumReplicas ", 7, pendingReplications.getNumReplicas(blk)); for (int i = 0; i < 7; i++) { // removes all replicas - pendingReplications.decrement(blk, genDatanodeId(i)); + pendingReplications.decrement(blk, storages[i].getDatanodeDescriptor()); } assertTrue(pendingReplications.size() == 9); - pendingReplications.increment(blk, genDatanodes(8)); + pendingReplications.increment(blk, + DatanodeStorageInfo.toDatanodeDescriptors( + DFSTestUtil.createDatanodeStorageInfos(8))); assertTrue(pendingReplications.size() == 10); // @@ -128,7 +117,9 @@ public class TestPendingReplication { for (int i = 10; i < 15; i++) { Block block = new Block(i, i, 0); - pendingReplications.increment(block, genDatanodes(i)); + pendingReplications.increment(block, + DatanodeStorageInfo.toDatanodeDescriptors( + DFSTestUtil.createDatanodeStorageInfos(i))); } assertTrue(pendingReplications.size() == 15); @@ -210,7 +201,7 @@ public class TestPendingReplication { DatanodeRegistration dnR = datanodes.get(i).getDNRegistrationForBP( poolId); StorageReceivedDeletedBlocks[] report = { - new StorageReceivedDeletedBlocks(dnR.getStorageID(), + new StorageReceivedDeletedBlocks("Fake-storage-ID-Ignored", new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo( blocks[0], BlockStatus.RECEIVED_BLOCK, "") }) }; cluster.getNameNodeRpc().blockReceivedAndDeleted(dnR, poolId, report); @@ -227,7 +218,7 @@ public class TestPendingReplication { DatanodeRegistration dnR = datanodes.get(i).getDNRegistrationForBP( poolId); StorageReceivedDeletedBlocks[] report = - { new StorageReceivedDeletedBlocks(dnR.getStorageID(), + { new StorageReceivedDeletedBlocks("Fake-storage-ID-Ignored", new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo( blocks[0], BlockStatus.RECEIVED_BLOCK, "") }) }; cluster.getNameNodeRpc().blockReceivedAndDeleted(dnR, poolId, report); @@ -291,9 +282,9 @@ public class TestPendingReplication { cluster.getNamesystem().writeLock(); try { bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0], - "TEST"); + "STORAGE_ID", "TEST"); bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[1], - "TEST"); + "STORAGE_ID", "TEST"); } finally { cluster.getNamesystem().writeUnlock(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java index a848a3323ce..3efeba2e4fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java @@ -21,6 +21,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.File; import java.util.ArrayList; @@ -32,6 +35,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -40,11 +44,14 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.LogVerificationAppender; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.namenode.Namesystem; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.Node; import org.apache.hadoop.test.PathUtils; @@ -58,6 +65,10 @@ import org.junit.Test; import org.junit.rules.ExpectedException; public class TestReplicationPolicy { + { + ((Log4JLogger)BlockPlacementPolicy.LOG).getLogger().setLevel(Level.ALL); + } + private Random random = DFSUtil.getRandom(); private static final int BLOCK_SIZE = 1024; private static final int NUM_OF_DATANODES = 6; @@ -66,6 +77,7 @@ public class TestReplicationPolicy { private static BlockPlacementPolicy replicator; private static final String filename = "/dummyfile.txt"; private static DatanodeDescriptor dataNodes[]; + private static DatanodeStorageInfo[] storages; // The interval for marking a datanode as stale, private static long staleInterval = DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT; @@ -73,17 +85,28 @@ public class TestReplicationPolicy { @Rule public ExpectedException exception = ExpectedException.none(); + private static void updateHeartbeatWithUsage(DatanodeDescriptor dn, + long capacity, long dfsUsed, long remaining, long blockPoolUsed, + int xceiverCount, int volFailures) { + dn.getStorageInfos()[0].setUtilizationForTesting( + capacity, dfsUsed, remaining, blockPoolUsed); + dn.updateHeartbeat( + BlockManagerTestUtil.getStorageReportsForDatanode(dn), + xceiverCount, volFailures); + } + @BeforeClass public static void setupCluster() throws Exception { Configuration conf = new HdfsConfiguration(); - dataNodes = new DatanodeDescriptor[] { - DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1"), - DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1"), - DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r2"), - DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2"), - DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d2/r3"), - DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3") - }; + final String[] racks = { + "/d1/r1", + "/d1/r1", + "/d1/r2", + "/d1/r2", + "/d2/r3", + "/d2/r3"}; + storages = DFSTestUtil.createDatanodeStorageInfos(racks); + dataNodes = DFSTestUtil.toDatanodeDescriptor(storages); FileSystem.setDefaultUri(conf, "hdfs://localhost:0"); conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); @@ -108,12 +131,19 @@ public class TestReplicationPolicy { dataNodes[i]); } for (int i=0; i < NUM_OF_DATANODES; i++) { - dataNodes[i].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[i], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } } + private static boolean isOnSameRack(DatanodeStorageInfo left, DatanodeStorageInfo right) { + return isOnSameRack(left, right.getDatanodeDescriptor()); + } + + private static boolean isOnSameRack(DatanodeStorageInfo left, DatanodeDescriptor right) { + return cluster.isOnSameRack(left.getDatanodeDescriptor(), right); + } /** * In this testcase, client is dataNodes[0]. So the 1st replica should be * placed on dataNodes[0], the 2nd replica should be placed on @@ -125,73 +155,73 @@ public class TestReplicationPolicy { */ @Test public void testChooseTarget1() throws Exception { - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0); assertEquals(targets.length, 0); targets = chooseTarget(1); assertEquals(targets.length, 1); - assertEquals(targets[0], dataNodes[0]); + assertEquals(storages[0], targets[0]); targets = chooseTarget(2); assertEquals(targets.length, 2); - assertEquals(targets[0], dataNodes[0]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[0], targets[0]); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3); assertEquals(targets.length, 3); - assertEquals(targets[0], dataNodes[0]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); + assertEquals(storages[0], targets[0]); + assertFalse(isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(targets[1], targets[2])); targets = chooseTarget(4); assertEquals(targets.length, 4); - assertEquals(targets[0], dataNodes[0]); - assertTrue(cluster.isOnSameRack(targets[1], targets[2]) || - cluster.isOnSameRack(targets[2], targets[3])); - assertFalse(cluster.isOnSameRack(targets[0], targets[2])); + assertEquals(storages[0], targets[0]); + assertTrue(isOnSameRack(targets[1], targets[2]) || + isOnSameRack(targets[2], targets[3])); + assertFalse(isOnSameRack(targets[0], targets[2])); - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } - private static DatanodeDescriptor[] chooseTarget(int numOfReplicas) { + private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas) { return chooseTarget(numOfReplicas, dataNodes[0]); } - private static DatanodeDescriptor[] chooseTarget(int numOfReplicas, + private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas, DatanodeDescriptor writer) { return chooseTarget(numOfReplicas, writer, - new ArrayList()); + new ArrayList()); } - private static DatanodeDescriptor[] chooseTarget(int numOfReplicas, - List chosenNodes) { + private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas, + List chosenNodes) { return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes); } - private static DatanodeDescriptor[] chooseTarget(int numOfReplicas, - DatanodeDescriptor writer, List chosenNodes) { + private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas, + DatanodeDescriptor writer, List chosenNodes) { return chooseTarget(numOfReplicas, writer, chosenNodes, null); } - private static DatanodeDescriptor[] chooseTarget(int numOfReplicas, - List chosenNodes, Set excludedNodes) { + private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas, + List chosenNodes, Set excludedNodes) { return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes, excludedNodes); } - private static DatanodeDescriptor[] chooseTarget( + private static DatanodeStorageInfo[] chooseTarget( int numOfReplicas, DatanodeDescriptor writer, - List chosenNodes, + List chosenNodes, Set excludedNodes) { return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes, - false, excludedNodes, BLOCK_SIZE); + false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); } /** @@ -205,8 +235,8 @@ public class TestReplicationPolicy { @Test public void testChooseTarget2() throws Exception { Set excludedNodes; - DatanodeDescriptor[] targets; - List chosenNodes = new ArrayList(); + DatanodeStorageInfo[] targets; + List chosenNodes = new ArrayList(); excludedNodes = new HashSet(); excludedNodes.add(dataNodes[1]); @@ -218,49 +248,52 @@ public class TestReplicationPolicy { excludedNodes.add(dataNodes[1]); targets = chooseTarget(1, chosenNodes, excludedNodes); assertEquals(targets.length, 1); - assertEquals(targets[0], dataNodes[0]); + assertEquals(storages[0], targets[0]); excludedNodes.clear(); chosenNodes.clear(); excludedNodes.add(dataNodes[1]); targets = chooseTarget(2, chosenNodes, excludedNodes); assertEquals(targets.length, 2); - assertEquals(targets[0], dataNodes[0]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[0], targets[0]); + + assertFalse(isOnSameRack(targets[0], targets[1])); excludedNodes.clear(); chosenNodes.clear(); excludedNodes.add(dataNodes[1]); targets = chooseTarget(3, chosenNodes, excludedNodes); assertEquals(targets.length, 3); - assertEquals(targets[0], dataNodes[0]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); + assertEquals(storages[0], targets[0]); + + assertFalse(isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(targets[1], targets[2])); excludedNodes.clear(); chosenNodes.clear(); excludedNodes.add(dataNodes[1]); targets = chooseTarget(4, chosenNodes, excludedNodes); assertEquals(targets.length, 4); - assertEquals(targets[0], dataNodes[0]); + assertEquals(storages[0], targets[0]); + for(int i=1; i<4; i++) { - assertFalse(cluster.isOnSameRack(targets[0], targets[i])); + assertFalse(isOnSameRack(targets[0], targets[i])); } - assertTrue(cluster.isOnSameRack(targets[1], targets[2]) || - cluster.isOnSameRack(targets[2], targets[3])); - assertFalse(cluster.isOnSameRack(targets[1], targets[3])); + assertTrue(isOnSameRack(targets[1], targets[2]) || + isOnSameRack(targets[2], targets[3])); + assertFalse(isOnSameRack(targets[1], targets[3])); excludedNodes.clear(); chosenNodes.clear(); excludedNodes.add(dataNodes[1]); - chosenNodes.add(dataNodes[2]); + chosenNodes.add(storages[2]); targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true, - excludedNodes, BLOCK_SIZE); + excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); System.out.println("targets=" + Arrays.asList(targets)); assertEquals(2, targets.length); //make sure that the chosen node is in the target. int i = 0; - for (; i < targets.length && !dataNodes[2].equals(targets[i]); i++); + for (; i < targets.length && !storages[2].equals(targets[i]); i++); assertTrue(i < targets.length); } @@ -275,40 +308,40 @@ public class TestReplicationPolicy { @Test public void testChooseTarget3() throws Exception { // make data node 0 to be not qualified to choose - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0); assertEquals(targets.length, 0); targets = chooseTarget(1); assertEquals(targets.length, 1); - assertEquals(targets[0], dataNodes[1]); + assertEquals(storages[1], targets[0]); targets = chooseTarget(2); assertEquals(targets.length, 2); - assertEquals(targets[0], dataNodes[1]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[1], targets[0]); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3); assertEquals(targets.length, 3); - assertEquals(targets[0], dataNodes[1]); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[1], targets[0]); + assertTrue(isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(4); assertEquals(targets.length, 4); - assertEquals(targets[0], dataNodes[1]); + assertEquals(storages[1], targets[0]); for(int i=1; i<4; i++) { - assertFalse(cluster.isOnSameRack(targets[0], targets[i])); + assertFalse(isOnSameRack(targets[0], targets[i])); } - assertTrue(cluster.isOnSameRack(targets[1], targets[2]) || - cluster.isOnSameRack(targets[2], targets[3])); - assertFalse(cluster.isOnSameRack(targets[1], targets[3])); + assertTrue(isOnSameRack(targets[1], targets[2]) || + isOnSameRack(targets[2], targets[3])); + assertFalse(isOnSameRack(targets[1], targets[3])); - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } @@ -325,35 +358,35 @@ public class TestReplicationPolicy { public void testChoooseTarget4() throws Exception { // make data node 0 & 1 to be not qualified to choose: not enough disk space for(int i=0; i<2; i++) { - dataNodes[i].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[i], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); } - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0); assertEquals(targets.length, 0); targets = chooseTarget(1); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); targets = chooseTarget(2); assertEquals(targets.length, 2); - assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3); assertEquals(targets.length, 3); for(int i=0; i<3; i++) { - assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0])); + assertFalse(isOnSameRack(targets[i], dataNodes[0])); } - assertTrue(cluster.isOnSameRack(targets[0], targets[1]) || - cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameRack(targets[0], targets[2])); + assertTrue(isOnSameRack(targets[0], targets[1]) || + isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameRack(targets[0], targets[2])); for(int i=0; i<2; i++) { - dataNodes[i].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[i], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } @@ -371,7 +404,7 @@ public class TestReplicationPolicy { DatanodeDescriptor writerDesc = DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r4"); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0, writerDesc); assertEquals(targets.length, 0); @@ -380,12 +413,12 @@ public class TestReplicationPolicy { targets = chooseTarget(2, writerDesc); assertEquals(targets.length, 2); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3, writerDesc); assertEquals(targets.length, 3); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameRack(targets[0], targets[1])); } /** @@ -415,7 +448,7 @@ public class TestReplicationPolicy { public void testChooseTargetWithMoreThanAvailableNodes() throws Exception { // make data node 0 & 1 to be not qualified to choose: not enough disk space for(int i=0; i<2; i++) { - dataNodes[i].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[i], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); } @@ -426,7 +459,7 @@ public class TestReplicationPolicy { // try to choose NUM_OF_DATANODES which is more than actually available // nodes. - DatanodeDescriptor[] targets = chooseTarget(NUM_OF_DATANODES); + DatanodeStorageInfo[] targets = chooseTarget(NUM_OF_DATANODES); assertEquals(targets.length, NUM_OF_DATANODES - 2); final List log = appender.getLog(); @@ -434,30 +467,42 @@ public class TestReplicationPolicy { assertFalse(log.size() == 0); final LoggingEvent lastLogEntry = log.get(log.size() - 1); - assertEquals(lastLogEntry.getLevel(), Level.WARN); + assertTrue(Level.WARN.isGreaterOrEqual(lastLogEntry.getLevel())); // Suppose to place replicas on each node but two data nodes are not // available for placing replica, so here we expect a short of 2 assertTrue(((String)lastLogEntry.getMessage()).contains("in need of 2")); for(int i=0; i<2; i++) { - dataNodes[i].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[i], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } } - private boolean containsWithinRange(DatanodeDescriptor target, + private boolean containsWithinRange(DatanodeStorageInfo target, DatanodeDescriptor[] nodes, int startIndex, int endIndex) { assert startIndex >= 0 && startIndex < nodes.length; assert endIndex >= startIndex && endIndex < nodes.length; for (int i = startIndex; i <= endIndex; i++) { - if (nodes[i].equals(target)) { + if (nodes[i].equals(target.getDatanodeDescriptor())) { return true; } } return false; } + private boolean containsWithinRange(DatanodeDescriptor target, + DatanodeStorageInfo[] nodes, int startIndex, int endIndex) { + assert startIndex >= 0 && startIndex < nodes.length; + assert endIndex >= startIndex && endIndex < nodes.length; + for (int i = startIndex; i <= endIndex; i++) { + if (nodes[i].getDatanodeDescriptor().equals(target)) { + return true; + } + } + return false; + } + @Test public void testChooseTargetWithStaleNodes() throws Exception { // Set dataNodes[0] as stale @@ -466,19 +511,19 @@ public class TestReplicationPolicy { .getDatanodeManager().getHeartbeatManager().heartbeatCheck(); assertTrue(namenode.getNamesystem().getBlockManager() .getDatanodeManager().shouldAvoidStaleDataNodesForWrite()); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; // We set the datanode[0] as stale, thus should choose datanode[1] since // datanode[1] is on the same rack with datanode[0] (writer) targets = chooseTarget(1); assertEquals(targets.length, 1); - assertEquals(targets[0], dataNodes[1]); + assertEquals(storages[1], targets[0]); Set excludedNodes = new HashSet(); excludedNodes.add(dataNodes[1]); - List chosenNodes = new ArrayList(); + List chosenNodes = new ArrayList(); targets = chooseTarget(1, chosenNodes, excludedNodes); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); // reset dataNodes[0].setLastUpdate(Time.now()); @@ -503,7 +548,7 @@ public class TestReplicationPolicy { namenode.getNamesystem().getBlockManager() .getDatanodeManager().getHeartbeatManager().heartbeatCheck(); - DatanodeDescriptor[] targets = chooseTarget(0); + DatanodeStorageInfo[] targets = chooseTarget(0); assertEquals(targets.length, 0); // Since we have 6 datanodes total, stale nodes should @@ -575,11 +620,12 @@ public class TestReplicationPolicy { .getDatanode(miniCluster.getDataNodes().get(0).getDatanodeId()); BlockPlacementPolicy replicator = miniCluster.getNameNode() .getNamesystem().getBlockManager().getBlockPlacementPolicy(); - DatanodeDescriptor[] targets = replicator.chooseTarget(filename, 3, - staleNodeInfo, new ArrayList(), false, null, BLOCK_SIZE); + DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 3, + staleNodeInfo, new ArrayList(), false, null, + BLOCK_SIZE, StorageType.DEFAULT); assertEquals(targets.length, 3); - assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo)); + assertFalse(isOnSameRack(targets[0], staleNodeInfo)); // Step 2. Set more than half of the datanodes as stale for (int i = 0; i < 4; i++) { @@ -600,10 +646,11 @@ public class TestReplicationPolicy { assertFalse(miniCluster.getNameNode().getNamesystem().getBlockManager() .getDatanodeManager().shouldAvoidStaleDataNodesForWrite()); // Call chooseTarget - targets = replicator.chooseTarget(filename, 3, - staleNodeInfo, new ArrayList(), false, null, BLOCK_SIZE); + targets = replicator.chooseTarget(filename, 3, staleNodeInfo, + new ArrayList(), false, null, BLOCK_SIZE, + StorageType.DEFAULT); assertEquals(targets.length, 3); - assertTrue(cluster.isOnSameRack(targets[0], staleNodeInfo)); + assertTrue(isOnSameRack(targets[0], staleNodeInfo)); // Step 3. Set 2 stale datanodes back to healthy nodes, // still have 2 stale nodes @@ -625,7 +672,7 @@ public class TestReplicationPolicy { // Call chooseTarget targets = chooseTarget(3, staleNodeInfo); assertEquals(targets.length, 3); - assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo)); + assertFalse(isOnSameRack(targets[0], staleNodeInfo)); } finally { miniCluster.shutdown(); } @@ -640,26 +687,26 @@ public class TestReplicationPolicy { */ @Test public void testRereplicate1() throws Exception { - List chosenNodes = new ArrayList(); - chosenNodes.add(dataNodes[0]); - DatanodeDescriptor[] targets; + List chosenNodes = new ArrayList(); + chosenNodes.add(storages[0]); + DatanodeStorageInfo[] targets; targets = chooseTarget(0, chosenNodes); assertEquals(targets.length, 0); targets = chooseTarget(1, chosenNodes); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); targets = chooseTarget(2, chosenNodes); assertEquals(targets.length, 2); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3, chosenNodes); assertEquals(targets.length, 3); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(targets[0], targets[2])); + assertTrue(isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[0], targets[2])); } /** @@ -671,22 +718,22 @@ public class TestReplicationPolicy { */ @Test public void testRereplicate2() throws Exception { - List chosenNodes = new ArrayList(); - chosenNodes.add(dataNodes[0]); - chosenNodes.add(dataNodes[1]); + List chosenNodes = new ArrayList(); + chosenNodes.add(storages[0]); + chosenNodes.add(storages[1]); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0, chosenNodes); assertEquals(targets.length, 0); targets = chooseTarget(1, chosenNodes); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); targets = chooseTarget(2, chosenNodes); assertEquals(targets.length, 2); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[1])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[1], dataNodes[0])); } /** @@ -698,31 +745,31 @@ public class TestReplicationPolicy { */ @Test public void testRereplicate3() throws Exception { - List chosenNodes = new ArrayList(); - chosenNodes.add(dataNodes[0]); - chosenNodes.add(dataNodes[2]); + List chosenNodes = new ArrayList(); + chosenNodes.add(storages[0]); + chosenNodes.add(storages[2]); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0, chosenNodes); assertEquals(targets.length, 0); targets = chooseTarget(1, chosenNodes); assertEquals(targets.length, 1); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(dataNodes[2], targets[0])); + assertTrue(isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(targets[0], dataNodes[2])); targets = chooseTarget(1, dataNodes[2], chosenNodes); assertEquals(targets.length, 1); - assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0])); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertTrue(isOnSameRack(targets[0], dataNodes[2])); + assertFalse(isOnSameRack(targets[0], dataNodes[0])); targets = chooseTarget(2, chosenNodes); assertEquals(targets.length, 2); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertTrue(isOnSameRack(targets[0], dataNodes[0])); targets = chooseTarget(2, dataNodes[2], chosenNodes); assertEquals(targets.length, 2); - assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0])); + assertTrue(isOnSameRack(targets[0], dataNodes[2])); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java index 297461bf935..283c36dd867 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.net.NetworkTopology; @@ -54,41 +55,57 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { private BlockPlacementPolicy replicator; private static final String filename = "/dummyfile.txt"; - private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] { - DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/n2"), - DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2/n3"), - DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/n3"), - DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/n4"), - DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3/n5"), - DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3/n6") - }; - - private final static DatanodeDescriptor dataNodesInBoundaryCase[] = - new DatanodeDescriptor[] { - DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r1/n2"), - DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/n3"), - DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/n3") - }; - - private final static DatanodeDescriptor dataNodesInMoreTargetsCase[] = - new DatanodeDescriptor[] { - DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/r1/n1"), - DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/r1/n2"), - DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/r1/n2"), - DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/r1/n3"), - DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/r1/n3"), - DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/r2/n4"), - DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/r2/n4"), - DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/r2/n5"), - DFSTestUtil.getDatanodeDescriptor("10.10.10.10", "/r2/n5"), - DFSTestUtil.getDatanodeDescriptor("11.11.11.11", "/r2/n6"), - DFSTestUtil.getDatanodeDescriptor("12.12.12.12", "/r2/n6"), + private static final DatanodeStorageInfo[] storages; + private static final DatanodeDescriptor[] dataNodes; + static { + final String[] racks = { + "/d1/r1/n1", + "/d1/r1/n1", + "/d1/r1/n2", + "/d1/r2/n3", + "/d1/r2/n3", + "/d1/r2/n4", + "/d2/r3/n5", + "/d2/r3/n6" + }; + storages = DFSTestUtil.createDatanodeStorageInfos(racks); + dataNodes = DFSTestUtil.toDatanodeDescriptor(storages); + } + + private static final DatanodeStorageInfo[] storagesInBoundaryCase; + private static final DatanodeDescriptor[] dataNodesInBoundaryCase; + static { + final String[] racksInBoundaryCase = { + "/d1/r1/n1", + "/d1/r1/n1", + "/d1/r1/n1", + "/d1/r1/n2", + "/d1/r2/n3", + "/d1/r2/n3" + }; + storagesInBoundaryCase = DFSTestUtil.createDatanodeStorageInfos(racksInBoundaryCase); + dataNodesInBoundaryCase = DFSTestUtil.toDatanodeDescriptor(storagesInBoundaryCase); + } + + private static final DatanodeStorageInfo[] storagesInMoreTargetsCase; + private final static DatanodeDescriptor[] dataNodesInMoreTargetsCase; + static { + final String[] racksInMoreTargetsCase = { + "/r1/n1", + "/r1/n1", + "/r1/n2", + "/r1/n2", + "/r1/n3", + "/r1/n3", + "/r2/n4", + "/r2/n4", + "/r2/n5", + "/r2/n5", + "/r2/n6", + "/r2/n6" + }; + storagesInMoreTargetsCase = DFSTestUtil.createDatanodeStorageInfos(racksInMoreTargetsCase); + dataNodesInMoreTargetsCase = DFSTestUtil.toDatanodeDescriptor(storagesInMoreTargetsCase); }; private final static DatanodeDescriptor NODE = @@ -126,9 +143,19 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { namenode.stop(); } + private static void updateHeartbeatWithUsage(DatanodeDescriptor dn, + long capacity, long dfsUsed, long remaining, long blockPoolUsed, + int xceiverCount, int volFailures) { + dn.getStorageInfos()[0].setUtilizationForTesting( + capacity, dfsUsed, remaining, blockPoolUsed); + dn.updateHeartbeat( + BlockManagerTestUtil.getStorageReportsForDatanode(dn), + xceiverCount, volFailures); + } + private static void setupDataNodeCapacity() { for(int i=0; i targetSet = new HashSet(); - for(DatanodeDescriptor node:targets) { + for(DatanodeStorageInfo storage:targets) { + final DatanodeDescriptor node = storage.getDatanodeDescriptor(); String nodeGroup = NetworkTopology.getLastHalf(node.getNetworkLocation()); if(targetSet.contains(nodeGroup)) { return false; @@ -153,34 +181,50 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { } return true; } - - private DatanodeDescriptor[] chooseTarget(int numOfReplicas) { + + private boolean isOnSameRack(DatanodeStorageInfo left, DatanodeStorageInfo right) { + return isOnSameRack(left.getDatanodeDescriptor(), right); + } + + private boolean isOnSameRack(DatanodeDescriptor left, DatanodeStorageInfo right) { + return cluster.isOnSameRack(left, right.getDatanodeDescriptor()); + } + + private boolean isOnSameNodeGroup(DatanodeStorageInfo left, DatanodeStorageInfo right) { + return isOnSameNodeGroup(left.getDatanodeDescriptor(), right); + } + + private boolean isOnSameNodeGroup(DatanodeDescriptor left, DatanodeStorageInfo right) { + return cluster.isOnSameNodeGroup(left, right.getDatanodeDescriptor()); + } + + private DatanodeStorageInfo[] chooseTarget(int numOfReplicas) { return chooseTarget(numOfReplicas, dataNodes[0]); } - private DatanodeDescriptor[] chooseTarget(int numOfReplicas, + private DatanodeStorageInfo[] chooseTarget(int numOfReplicas, DatanodeDescriptor writer) { return chooseTarget(numOfReplicas, writer, - new ArrayList()); + new ArrayList()); } - private DatanodeDescriptor[] chooseTarget(int numOfReplicas, - List chosenNodes) { + private DatanodeStorageInfo[] chooseTarget(int numOfReplicas, + List chosenNodes) { return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes); } - private DatanodeDescriptor[] chooseTarget(int numOfReplicas, - DatanodeDescriptor writer, List chosenNodes) { + private DatanodeStorageInfo[] chooseTarget(int numOfReplicas, + DatanodeDescriptor writer, List chosenNodes) { return chooseTarget(numOfReplicas, writer, chosenNodes, null); } - private DatanodeDescriptor[] chooseTarget( + private DatanodeStorageInfo[] chooseTarget( int numOfReplicas, DatanodeDescriptor writer, - List chosenNodes, + List chosenNodes, Set excludedNodes) { return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes, - false, excludedNodes, BLOCK_SIZE); + false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); } /** @@ -193,48 +237,52 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { * @throws Exception */ public void testChooseTarget1() throws Exception { - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0); assertEquals(targets.length, 0); targets = chooseTarget(1); assertEquals(targets.length, 1); - assertEquals(targets[0], dataNodes[0]); + assertEquals(storages[0], targets[0]); + targets = chooseTarget(2); assertEquals(targets.length, 2); - assertEquals(targets[0], dataNodes[0]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[0], targets[0]); + + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3); assertEquals(targets.length, 3); - assertEquals(targets[0], dataNodes[0]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameNodeGroup(targets[1], targets[2])); + assertEquals(storages[0], targets[0]); + + assertFalse(isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameNodeGroup(targets[1], targets[2])); targets = chooseTarget(4); assertEquals(targets.length, 4); - assertEquals(targets[0], dataNodes[0]); - assertTrue(cluster.isOnSameRack(targets[1], targets[2]) || - cluster.isOnSameRack(targets[2], targets[3])); - assertFalse(cluster.isOnSameRack(targets[0], targets[2])); + assertEquals(storages[0], targets[0]); + + assertTrue(isOnSameRack(targets[1], targets[2]) || + isOnSameRack(targets[2], targets[3])); + assertFalse(isOnSameRack(targets[0], targets[2])); // Make sure no more than one replicas are on the same nodegroup verifyNoTwoTargetsOnSameNodeGroup(targets); - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } - private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeDescriptor[] targets) { + private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeStorageInfo[] targets) { Set nodeGroupSet = new HashSet(); - for (DatanodeDescriptor target: targets) { - nodeGroupSet.add(target.getNetworkLocation()); + for (DatanodeStorageInfo target: targets) { + nodeGroupSet.add(target.getDatanodeDescriptor().getNetworkLocation()); } assertEquals(nodeGroupSet.size(), targets.length); } @@ -248,36 +296,37 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { * @throws Exception */ public void testChooseTarget2() throws Exception { - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator; - List chosenNodes = new ArrayList(); + List chosenNodes = new ArrayList(); Set excludedNodes = new HashSet(); excludedNodes.add(dataNodes[1]); targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false, - excludedNodes, BLOCK_SIZE); + excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); assertEquals(targets.length, 4); - assertEquals(targets[0], dataNodes[0]); + assertEquals(storages[0], targets[0]); + assertTrue(cluster.isNodeGroupAware()); // Make sure no replicas are on the same nodegroup for (int i=1;i<4;i++) { - assertFalse(cluster.isOnSameNodeGroup(targets[0], targets[i])); + assertFalse(isOnSameNodeGroup(targets[0], targets[i])); } - assertTrue(cluster.isOnSameRack(targets[1], targets[2]) || - cluster.isOnSameRack(targets[2], targets[3])); - assertFalse(cluster.isOnSameRack(targets[1], targets[3])); + assertTrue(isOnSameRack(targets[1], targets[2]) || + isOnSameRack(targets[2], targets[3])); + assertFalse(isOnSameRack(targets[1], targets[3])); excludedNodes.clear(); chosenNodes.clear(); excludedNodes.add(dataNodes[1]); - chosenNodes.add(dataNodes[2]); + chosenNodes.add(storages[2]); targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true, - excludedNodes, BLOCK_SIZE); + excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); System.out.println("targets=" + Arrays.asList(targets)); assertEquals(2, targets.length); //make sure that the chosen node is in the target. int i = 0; - for(; i < targets.length && !dataNodes[2].equals(targets[i]); i++); + for(; i < targets.length && !storages[2].equals(targets[i]); i++); assertTrue(i < targets.length); } @@ -291,38 +340,38 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { */ public void testChooseTarget3() throws Exception { // make data node 0 to be not qualified to choose - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0); assertEquals(targets.length, 0); targets = chooseTarget(1); assertEquals(targets.length, 1); - assertEquals(targets[0], dataNodes[1]); + assertEquals(storages[1], targets[0]); targets = chooseTarget(2); assertEquals(targets.length, 2); - assertEquals(targets[0], dataNodes[1]); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[1], targets[0]); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3); assertEquals(targets.length, 3); - assertEquals(targets[0], dataNodes[1]); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertEquals(storages[1], targets[0]); + assertTrue(isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(4); assertEquals(targets.length, 4); - assertEquals(targets[0], dataNodes[1]); + assertEquals(storages[1], targets[0]); assertTrue(cluster.isNodeGroupAware()); verifyNoTwoTargetsOnSameNodeGroup(targets); - assertTrue(cluster.isOnSameRack(targets[1], targets[2]) || - cluster.isOnSameRack(targets[2], targets[3])); + assertTrue(isOnSameRack(targets[1], targets[2]) || + isOnSameRack(targets[2], targets[3])); - dataNodes[0].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[0], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); } @@ -339,33 +388,33 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { public void testChooseTarget4() throws Exception { // make data node 0-2 to be not qualified to choose: not enough disk space for(int i=0; i<3; i++) { - dataNodes[i].updateHeartbeat( + updateHeartbeatWithUsage(dataNodes[i], 2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); } - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0); assertEquals(targets.length, 0); targets = chooseTarget(1); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0])); + assertFalse(isOnSameRack(dataNodes[0], targets[0])); targets = chooseTarget(2); assertEquals(targets.length, 2); - assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertFalse(isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3); assertEquals(targets.length, 3); for(int i=0; i<3; i++) { - assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0])); + assertFalse(isOnSameRack(dataNodes[0], targets[i])); } verifyNoTwoTargetsOnSameNodeGroup(targets); - assertTrue(cluster.isOnSameRack(targets[0], targets[1]) || - cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameRack(targets[0], targets[2])); + assertTrue(isOnSameRack(targets[0], targets[1]) || + isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameRack(targets[0], targets[2])); } /** @@ -377,7 +426,7 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { */ public void testChooseTarget5() throws Exception { setupDataNodeCapacity(); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0, NODE); assertEquals(targets.length, 0); @@ -386,12 +435,12 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { targets = chooseTarget(2, NODE); assertEquals(targets.length, 2); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3, NODE); assertEquals(targets.length, 3); - assertTrue(cluster.isOnSameRack(targets[1], targets[2])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(targets[1], targets[2])); + assertFalse(isOnSameRack(targets[0], targets[1])); verifyNoTwoTargetsOnSameNodeGroup(targets); } @@ -404,27 +453,27 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { */ public void testRereplicate1() throws Exception { setupDataNodeCapacity(); - List chosenNodes = new ArrayList(); - chosenNodes.add(dataNodes[0]); - DatanodeDescriptor[] targets; + List chosenNodes = new ArrayList(); + chosenNodes.add(storages[0]); + DatanodeStorageInfo[] targets; targets = chooseTarget(0, chosenNodes); assertEquals(targets.length, 0); targets = chooseTarget(1, chosenNodes); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(dataNodes[0], targets[0])); targets = chooseTarget(2, chosenNodes); assertEquals(targets.length, 2); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(targets[0], targets[1])); + assertTrue(isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(targets[0], targets[1])); targets = chooseTarget(3, chosenNodes); assertEquals(targets.length, 3); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(targets[0], targets[2])); + assertTrue(isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameNodeGroup(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(targets[0], targets[2])); } /** @@ -436,22 +485,22 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { */ public void testRereplicate2() throws Exception { setupDataNodeCapacity(); - List chosenNodes = new ArrayList(); - chosenNodes.add(dataNodes[0]); - chosenNodes.add(dataNodes[1]); + List chosenNodes = new ArrayList(); + chosenNodes.add(storages[0]); + chosenNodes.add(storages[1]); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0, chosenNodes); assertEquals(targets.length, 0); targets = chooseTarget(1, chosenNodes); assertEquals(targets.length, 1); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(dataNodes[0], targets[0])); targets = chooseTarget(2, chosenNodes); assertEquals(targets.length, 2); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]) && - cluster.isOnSameRack(dataNodes[0], targets[1])); + assertFalse(isOnSameRack(dataNodes[0], targets[0]) && + isOnSameRack(dataNodes[0], targets[1])); } /** @@ -463,33 +512,33 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { */ public void testRereplicate3() throws Exception { setupDataNodeCapacity(); - List chosenNodes = new ArrayList(); - chosenNodes.add(dataNodes[0]); - chosenNodes.add(dataNodes[3]); + List chosenNodes = new ArrayList(); + chosenNodes.add(storages[0]); + chosenNodes.add(storages[3]); - DatanodeDescriptor[] targets; + DatanodeStorageInfo[] targets; targets = chooseTarget(0, chosenNodes); assertEquals(targets.length, 0); targets = chooseTarget(1, chosenNodes); assertEquals(targets.length, 1); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameRack(dataNodes[3], targets[0])); + assertTrue(isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameRack(dataNodes[3], targets[0])); targets = chooseTarget(1, dataNodes[3], chosenNodes); assertEquals(targets.length, 1); - assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0])); - assertFalse(cluster.isOnSameNodeGroup(dataNodes[3], targets[0])); - assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0])); + assertTrue(isOnSameRack(dataNodes[3], targets[0])); + assertFalse(isOnSameNodeGroup(dataNodes[3], targets[0])); + assertFalse(isOnSameRack(dataNodes[0], targets[0])); targets = chooseTarget(2, chosenNodes); assertEquals(targets.length, 2); - assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0])); - assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0])); + assertTrue(isOnSameRack(dataNodes[0], targets[0])); + assertFalse(isOnSameNodeGroup(dataNodes[0], targets[0])); targets = chooseTarget(2, dataNodes[3], chosenNodes); assertEquals(targets.length, 2); - assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0])); + assertTrue(isOnSameRack(dataNodes[3], targets[0])); } /** @@ -563,16 +612,17 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { cluster.add(dataNodesInBoundaryCase[i]); } for(int i=0; i chosenNodes = new ArrayList(); - chosenNodes.add(dataNodesInBoundaryCase[0]); - chosenNodes.add(dataNodesInBoundaryCase[5]); - DatanodeDescriptor[] targets; + List chosenNodes = new ArrayList(); + chosenNodes.add(storagesInBoundaryCase[0]); + chosenNodes.add(storagesInBoundaryCase[5]); + DatanodeStorageInfo[] targets; targets = chooseTarget(1, dataNodesInBoundaryCase[0], chosenNodes); - assertFalse(cluster.isOnSameNodeGroup(targets[0], - dataNodesInBoundaryCase[0])); - assertFalse(cluster.isOnSameNodeGroup(targets[0], - dataNodesInBoundaryCase[5])); + assertFalse(isOnSameNodeGroup(dataNodesInBoundaryCase[0], targets[0])); + assertFalse(isOnSameNodeGroup(dataNodesInBoundaryCase[5], targets[0])); assertTrue(checkTargetsOnDifferentNodeGroup(targets)); } @@ -638,12 +686,12 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase { } for(int i=0; i live = new ArrayList(); live.add(dnDesc1); live.add(dnDesc2); @@ -598,3 +609,4 @@ public class TestJspHelper { MessageFormat.format(EXPECTED_NOTF_PATTERN, version))); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java index 274e5a3a6bd..30a7f4dbee9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java @@ -23,10 +23,10 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import javax.management.NotCompliantMBeanException; import javax.management.ObjectName; @@ -48,7 +48,9 @@ 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.metrics.FSDatasetMBean; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.util.DataChecksum; @@ -74,7 +76,7 @@ public class SimulatedFSDataset implements FsDatasetSpi { @Override public SimulatedFSDataset newInstance(DataNode datanode, DataStorage storage, Configuration conf) throws IOException { - return new SimulatedFSDataset(datanode, storage, conf); + return new SimulatedFSDataset(storage, conf); } @Override @@ -135,6 +137,11 @@ public class SimulatedFSDataset implements FsDatasetSpi { } } + @Override + public String getStorageUuid() { + return storage.getStorageUuid(); + } + @Override synchronized public long getGenerationStamp() { return theBlock.getGenerationStamp(); @@ -318,13 +325,15 @@ public class SimulatedFSDataset implements FsDatasetSpi { private static class SimulatedStorage { private Map map = new HashMap(); - private long capacity; // in bytes + private final String storageUuid = "SimulatedStroage-" + DatanodeStorage.generateUuid(); + + private final long capacity; // in bytes synchronized long getFree() { return capacity - getUsed(); } - synchronized long getCapacity() { + long getCapacity() { return capacity; } @@ -379,22 +388,33 @@ public class SimulatedFSDataset implements FsDatasetSpi { } return bpStorage; } + + String getStorageUuid() { + return storageUuid; + } + + synchronized StorageReport getStorageReport(String bpid) { + return new StorageReport(getStorageUuid(), false, getCapacity(), + getUsed(), getFree(), map.get(bpid).getUsed()); + } } private final Map> blockMap = new HashMap>(); private final SimulatedStorage storage; - private final String storageId; + private final String datanodeUuid; - public SimulatedFSDataset(DataNode datanode, DataStorage storage, - Configuration conf) { + public SimulatedFSDataset(DataStorage storage, Configuration conf) { if (storage != null) { - storage.createStorageID(datanode.getXferPort()); - this.storageId = storage.getStorageID(); + for (int i = 0; i < storage.getNumStorageDirs(); ++i) { + storage.createStorageID(storage.getStorageDir(i)); + } + this.datanodeUuid = storage.getDatanodeUuid(); } else { - this.storageId = "unknownStorageId" + new Random().nextInt(); + this.datanodeUuid = "SimulatedDatanode-" + DataNode.generateUuid(); } - registerMBean(storageId); + + registerMBean(datanodeUuid); this.storage = new SimulatedStorage( conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY)); } @@ -451,8 +471,7 @@ public class SimulatedFSDataset implements FsDatasetSpi { } } - @Override - public synchronized BlockListAsLongs getBlockReport(String bpid) { + synchronized BlockListAsLongs getBlockReport(String bpid) { final List blocks = new ArrayList(); final Map map = blockMap.get(bpid); if (map != null) { @@ -465,6 +484,12 @@ public class SimulatedFSDataset implements FsDatasetSpi { return new BlockListAsLongs(blocks, null); } + @Override + public synchronized Map getBlockReports( + String bpid) { + return Collections.singletonMap(new DatanodeStorage(storage.storageUuid), getBlockReport(bpid)); + } + @Override // FSDatasetMBean public long getCapacity() { return storage.getCapacity(); @@ -619,7 +644,7 @@ public class SimulatedFSDataset implements FsDatasetSpi { } @Override // FsDatasetSpi - public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen) + public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { final Map map = getMap(b.getBlockPoolId()); BInfo binfo = map.get(b.getLocalBlock()); @@ -633,6 +658,7 @@ public class SimulatedFSDataset implements FsDatasetSpi { map.remove(b.getLocalBlock()); binfo.theBlock.setGenerationStamp(newGS); map.put(binfo.theBlock, binfo); + return binfo.getStorageUuid(); } @Override // FsDatasetSpi @@ -889,7 +915,7 @@ public class SimulatedFSDataset implements FsDatasetSpi { @Override public String getStorageInfo() { - return "Simulated FSDataset-" + storageId; + return "Simulated FSDataset-" + datanodeUuid; } @Override @@ -916,7 +942,8 @@ public class SimulatedFSDataset implements FsDatasetSpi { public String updateReplicaUnderRecovery(ExtendedBlock oldBlock, long recoveryId, long newlength) { - return storageId; + // Caller does not care about the exact Storage UUID returned. + return datanodeUuid; } @Override // FsDatasetSpi @@ -970,11 +997,6 @@ public class SimulatedFSDataset implements FsDatasetSpi { throw new UnsupportedOperationException(); } - @Override - public String[] getBlockPoolList() { - throw new UnsupportedOperationException(); - } - @Override public void checkAndUpdate(String bpid, long blockId, File diskFile, File diskMetaFile, FsVolumeSpi vol) { @@ -987,7 +1009,12 @@ public class SimulatedFSDataset implements FsDatasetSpi { } @Override - public List getFinalizedBlocks(String bpid) { + public StorageReport[] getStorageReports(String bpid) { + return new StorageReport[] {storage.getStorageReport(bpid)}; + } + + @Override + public List getFinalizedBlocks(String bpid) { throw new UnsupportedOperationException(); } @@ -1006,3 +1033,4 @@ public class SimulatedFSDataset implements FsDatasetSpi { throw new UnsupportedOperationException(); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java index 23a10d4024f..2d4d0a774f8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java @@ -104,7 +104,7 @@ public class TestBPOfferService { .when(mockDn).getMetrics(); // Set up a simulated dataset with our fake BP - mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, null, conf)); + mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, conf)); mockFSDataset.addBlockPool(FAKE_BPID, conf); // Wire the dataset to the DN. @@ -178,7 +178,7 @@ public class TestBPOfferService { waitForBlockReport(mockNN2); // When we receive a block, it should report it to both NNs - bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, ""); + bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "", ""); ReceivedDeletedBlockInfo[] ret = waitForBlockReceived(FAKE_BLOCK, mockNN1); assertEquals(1, ret.length); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java index a400e850594..edf27a5b12b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java @@ -35,6 +35,7 @@ import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; +import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -120,7 +121,7 @@ public class TestBlockRecovery { * @throws IOException */ @Before - public void startUp() throws IOException { + public void startUp() throws IOException, URISyntaxException { conf = new HdfsConfiguration(); conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, DATA_DIR); conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0"); @@ -129,11 +130,12 @@ public class TestBlockRecovery { conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0); FileSystem.setDefaultUri(conf, "hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort()); - ArrayList dirs = new ArrayList(); + ArrayList locations = new ArrayList(); File dataDir = new File(DATA_DIR); FileUtil.fullyDelete(dataDir); dataDir.mkdirs(); - dirs.add(dataDir); + StorageLocation location = StorageLocation.parse(dataDir.getPath()); + locations.add(location); final DatanodeProtocolClientSideTranslatorPB namenode = mock(DatanodeProtocolClientSideTranslatorPB.class); @@ -159,7 +161,7 @@ public class TestBlockRecovery { new DatanodeCommand[0], new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1))); - dn = new DataNode(conf, dirs, null) { + dn = new DataNode(conf, locations, null) { @Override DatanodeProtocolClientSideTranslatorPB connectToNN( InetSocketAddress nnAddr) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java index a138071b525..02faa595e4c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java @@ -265,7 +265,7 @@ public class TestBlockReplacement { // sendRequest DataOutputStream out = new DataOutputStream(sock.getOutputStream()); new Sender(out).replaceBlock(block, BlockTokenSecretManager.DUMMY_TOKEN, - source.getStorageID(), sourceProxy); + source.getDatanodeUuid(), sourceProxy); out.flush(); // receiveResponse DataInputStream reply = new DataInputStream(sock.getInputStream()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java index 8f82e5400ce..1dbf207e4d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.hdfs.server.datanode; +import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import java.io.File; @@ -25,6 +27,7 @@ import java.io.FilenameFilter; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeoutException; @@ -88,7 +91,7 @@ public class TestBlockReport { private MiniDFSCluster cluster; private DistributedFileSystem fs; - Random rand = new Random(RAND_LIMIT); + private static Random rand = new Random(RAND_LIMIT); private static Configuration conf; @@ -112,6 +115,48 @@ public class TestBlockReport { cluster.shutdown(); } + // Generate a block report, optionally corrupting the generation + // stamp and/or length of one block. + private static StorageBlockReport[] getBlockReports( + DataNode dn, String bpid, boolean corruptOneBlockGs, + boolean corruptOneBlockLen) { + Map perVolumeBlockLists = + dn.getFSDataset().getBlockReports(bpid); + + // Send block report + StorageBlockReport[] reports = + new StorageBlockReport[perVolumeBlockLists.size()]; + boolean corruptedGs = false; + boolean corruptedLen = false; + + int reportIndex = 0; + for(Map.Entry kvPair : perVolumeBlockLists.entrySet()) { + DatanodeStorage dnStorage = kvPair.getKey(); + BlockListAsLongs blockList = kvPair.getValue(); + + // Walk the list of blocks until we find one each to corrupt the + // generation stamp and length, if so requested. + for (int i = 0; i < blockList.getNumberOfBlocks(); ++i) { + if (corruptOneBlockGs && !corruptedGs) { + blockList.corruptBlockGSForTesting(i, rand); + LOG.info("Corrupted the GS for block ID " + i); + corruptedGs = true; + } else if (corruptOneBlockLen && !corruptedLen) { + blockList.corruptBlockLengthForTesting(i, rand); + LOG.info("Corrupted the length for block ID " + i); + corruptedLen = true; + } else { + break; + } + } + + reports[reportIndex++] = + new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs()); + } + + return reports; + } + /** * Test write a file, verifies and closes it. Then the length of the blocks * are messed up and BlockReport is forced. @@ -152,10 +197,8 @@ public class TestBlockReport { DataNode dn = cluster.getDataNodes().get(DN_N0); String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; - cluster.getNameNodeRpc().blockReport(dnR, poolId, report); + StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false); + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); List blocksAfterReport = DFSTestUtil.getAllBlocks(fs.open(filePath)); @@ -210,7 +253,6 @@ public class TestBlockReport { for (Integer aRemovedIndex : removedIndex) { blocks2Remove.add(lBlocks.get(aRemovedIndex).getBlock()); } - ArrayList blocks = locatedToBlocks(lBlocks, removedIndex); if(LOG.isDebugEnabled()) { LOG.debug("Number of blocks allocated " + lBlocks.size()); @@ -224,8 +266,11 @@ public class TestBlockReport { for (File f : findAllFiles(dataDir, new MyFileFilter(b.getBlockName(), true))) { DataNodeTestUtils.getFSDataset(dn0).unfinalizeBlock(b); - if (!f.delete()) + if (!f.delete()) { LOG.warn("Couldn't delete " + b.getBlockName()); + } else { + LOG.debug("Deleted file " + f.toString()); + } } } @@ -234,10 +279,8 @@ public class TestBlockReport { // all blocks belong to the same file, hence same BP String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; - cluster.getNameNodeRpc().blockReport(dnR, poolId, report); + StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false); + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem() .getBlockManager()); @@ -252,9 +295,8 @@ public class TestBlockReport { /** - * Test writes a file and closes it. Then test finds a block - * and changes its GS to be < of original one. - * New empty block is added to the list of blocks. + * Test writes a file and closes it. + * Block reported is generated with a bad GS for a single block. * Block report is forced and the check for # of corrupted blocks is performed. * * @throws IOException in case of an error @@ -263,50 +305,64 @@ public class TestBlockReport { public void blockReport_03() throws IOException { final String METHOD_NAME = GenericTestUtils.getMethodName(); Path filePath = new Path("/" + METHOD_NAME + ".dat"); - - ArrayList blocks = - prepareForRide(filePath, METHOD_NAME, FILE_SIZE); - - // The block with modified GS won't be found. Has to be deleted - blocks.get(0).setGenerationStamp(rand.nextLong()); - // This new block is unknown to NN and will be mark for deletion. - blocks.add(new Block()); + ArrayList blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath); // all blocks belong to the same file, hence same BP DataNode dn = cluster.getDataNodes().get(DN_N0); String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; + StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false); DatanodeCommand dnCmd = - cluster.getNameNodeRpc().blockReport(dnR, poolId, report); + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); if(LOG.isDebugEnabled()) { LOG.debug("Got the command: " + dnCmd); } printStats(); - assertEquals("Wrong number of CorruptedReplica+PendingDeletion " + - "blocks is found", 2, - cluster.getNamesystem().getCorruptReplicaBlocks() + - cluster.getNamesystem().getPendingDeletionBlocks()); + assertThat("Wrong number of corrupt blocks", + cluster.getNamesystem().getCorruptReplicaBlocks(), is(1L)); + assertThat("Wrong number of PendingDeletion blocks", + cluster.getNamesystem().getPendingDeletionBlocks(), is(0L)); } /** - * This test isn't a representative case for BlockReport - * The empty method is going to be left here to keep the naming - * of the test plan in synch with the actual implementation + * Test writes a file and closes it. + * Block reported is generated with an extra block. + * Block report is forced and the check for # of pendingdeletion + * blocks is performed. + * + * @throws IOException in case of an error */ - public void blockReport_04() { - } + @Test + public void blockReport_04() throws IOException { + final String METHOD_NAME = GenericTestUtils.getMethodName(); + Path filePath = new Path("/" + METHOD_NAME + ".dat"); + DFSTestUtil.createFile(fs, filePath, + FILE_SIZE, REPL_FACTOR, rand.nextLong()); - // Client requests new block from NN. The test corrupts this very block - // and forces new block report. - // The test case isn't specific for BlockReport because it relies on - // BlockScanner which is out of scope of this test - // Keeping the name to be in synch with the test plan - // - public void blockReport_05() { + + DataNode dn = cluster.getDataNodes().get(DN_N0); + // all blocks belong to the same file, hence same BP + String poolId = cluster.getNamesystem().getBlockPoolId(); + + // Create a bogus new block which will not be present on the namenode. + ExtendedBlock b = new ExtendedBlock( + poolId, rand.nextLong(), 1024L, rand.nextLong()); + dn.getFSDataset().createRbw(b); + + DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); + StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false); + DatanodeCommand dnCmd = + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); + if(LOG.isDebugEnabled()) { + LOG.debug("Got the command: " + dnCmd); + } + printStats(); + + assertThat("Wrong number of corrupt blocks", + cluster.getNamesystem().getCorruptReplicaBlocks(), is(0L)); + assertThat("Wrong number of PendingDeletion blocks", + cluster.getNamesystem().getPendingDeletionBlocks(), is(1L)); } /** @@ -323,17 +379,15 @@ public class TestBlockReport { Path filePath = new Path("/" + METHOD_NAME + ".dat"); final int DN_N1 = DN_N0 + 1; - ArrayList blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath); + writeFile(METHOD_NAME, FILE_SIZE, filePath); startDNandWait(filePath, true); - // all blocks belong to the same file, hence same BP + // all blocks belong to the same file, hence same BP DataNode dn = cluster.getDataNodes().get(DN_N1); String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; - cluster.getNameNodeRpc().blockReport(dnR, poolId, report); + StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false); + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); printStats(); assertEquals("Wrong number of PendingReplication Blocks", 0, cluster.getNamesystem().getUnderReplicatedBlocks()); @@ -353,68 +407,40 @@ public class TestBlockReport { * @throws IOException in case of an error */ @Test - // Currently this test is failing as expected 'cause the correct behavior is - // not yet implemented (9/15/09) public void blockReport_07() throws Exception { final String METHOD_NAME = GenericTestUtils.getMethodName(); Path filePath = new Path("/" + METHOD_NAME + ".dat"); final int DN_N1 = DN_N0 + 1; // write file and start second node to be "older" than the original - ArrayList blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath); + writeFile(METHOD_NAME, FILE_SIZE, filePath); startDNandWait(filePath, true); - int randIndex = rand.nextInt(blocks.size()); - // Get a block and screw its GS - Block corruptedBlock = blocks.get(randIndex); - String secondNode = cluster.getDataNodes().get(DN_N1).getStorageId(); - if(LOG.isDebugEnabled()) { - LOG.debug("Working with " + secondNode); - LOG.debug("BlockGS before " + blocks.get(randIndex).getGenerationStamp()); - } - corruptBlockGS(corruptedBlock); - if(LOG.isDebugEnabled()) { - LOG.debug("BlockGS after " + blocks.get(randIndex).getGenerationStamp()); - LOG.debug("Done corrupting GS of " + corruptedBlock.getBlockName()); - } // all blocks belong to the same file, hence same BP DataNode dn = cluster.getDataNodes().get(DN_N1); String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; - cluster.getNameNodeRpc().blockReport(dnR, poolId, report); - printStats(); - assertEquals("Wrong number of Corrupted blocks", - 1, cluster.getNamesystem().getCorruptReplicaBlocks() + -// the following might have to be added into the equation if -// the same block could be in two different states at the same time -// and then the expected number of has to be changed to '2' -// cluster.getNamesystem().getPendingReplicationBlocks() + - cluster.getNamesystem().getPendingDeletionBlocks()); - - // Get another block and screw its length to be less than original - if (randIndex == 0) - randIndex++; - else - randIndex--; - corruptedBlock = blocks.get(randIndex); - corruptBlockLen(corruptedBlock); - if(LOG.isDebugEnabled()) { - LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName()); - } - - report[0] = new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()); - cluster.getNameNodeRpc().blockReport(dnR, poolId, report); + StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false); + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); printStats(); - assertEquals("Wrong number of Corrupted blocks", - 2, cluster.getNamesystem().getCorruptReplicaBlocks() + - cluster.getNamesystem().getPendingReplicationBlocks() + - cluster.getNamesystem().getPendingDeletionBlocks()); + assertThat("Wrong number of corrupt blocks", + cluster.getNamesystem().getCorruptReplicaBlocks(), is(1L)); + assertThat("Wrong number of PendingDeletion blocks", + cluster.getNamesystem().getPendingDeletionBlocks(), is(0L)); + assertThat("Wrong number of PendingReplication blocks", + cluster.getNamesystem().getPendingReplicationBlocks(), is(0L)); + + reports = getBlockReports(dn, poolId, true, true); + cluster.getNameNodeRpc().blockReport(dnR, poolId, reports); + printStats(); + + assertThat("Wrong number of corrupt blocks", + cluster.getNamesystem().getCorruptReplicaBlocks(), is(2L)); + assertThat("Wrong number of PendingDeletion blocks", + cluster.getNamesystem().getPendingDeletionBlocks(), is(0L)); + assertThat("Wrong number of PendingReplication blocks", + cluster.getNamesystem().getPendingReplicationBlocks(), is(0L)); printStats(); @@ -457,9 +483,7 @@ public class TestBlockReport { DataNode dn = cluster.getDataNodes().get(DN_N1); String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; + StorageBlockReport[] report = getBlockReports(dn, poolId, false, false); cluster.getNameNodeRpc().blockReport(dnR, poolId, report); printStats(); assertEquals("Wrong number of PendingReplication blocks", @@ -490,14 +514,11 @@ public class TestBlockReport { // write file and start second node to be "older" than the original try { - ArrayList blocks = - writeFile(METHOD_NAME, 12 * bytesChkSum, filePath); + writeFile(METHOD_NAME, 12 * bytesChkSum, filePath); Block bl = findBlock(filePath, 12 * bytesChkSum); BlockChecker bc = new BlockChecker(filePath); bc.start(); - corruptBlockGS(bl); - corruptBlockLen(bl); waitForTempReplica(bl, DN_N1); @@ -505,9 +526,7 @@ public class TestBlockReport { DataNode dn = cluster.getDataNodes().get(DN_N1); String poolId = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId); - StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - new BlockListAsLongs(blocks, null).getBlockListAsLongs()) }; + StorageBlockReport[] report = getBlockReports(dn, poolId, true, true); cluster.getNameNodeRpc().blockReport(dnR, poolId, report); printStats(); assertEquals("Wrong number of PendingReplication blocks", @@ -783,38 +802,6 @@ public class TestBlockReport { ((Log4JLogger) TestBlockReport.LOG).getLogger().setLevel(Level.ALL); } - private void corruptBlockLen(final Block block) - throws IOException { - if (block == null) { - throw new IOException("Block isn't suppose to be null"); - } - long oldLen = block.getNumBytes(); - long newLen = oldLen - rand.nextLong(); - assertTrue("Old and new length shouldn't be the same", - block.getNumBytes() != newLen); - block.setNumBytes(newLen); - if(LOG.isDebugEnabled()) { - LOG.debug("Length of " + block.getBlockName() + - " is changed to " + newLen + " from " + oldLen); - } - } - - private void corruptBlockGS(final Block block) - throws IOException { - if (block == null) { - throw new IOException("Block isn't suppose to be null"); - } - long oldGS = block.getGenerationStamp(); - long newGS = oldGS - rand.nextLong(); - assertTrue("Old and new GS shouldn't be the same", - block.getGenerationStamp() != newGS); - block.setGenerationStamp(newGS); - if(LOG.isDebugEnabled()) { - LOG.debug("Generation stamp of " + block.getBlockName() + - " is changed to " + block.getGenerationStamp() + " from " + oldGS); - } - } - private Block findBlock(Path path, long size) throws IOException { Block ret; List lbs = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java index fc7a3ff01d1..53babb471db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java @@ -19,12 +19,14 @@ package org.apache.hadoop.hdfs.server.datanode; import java.io.*; -import java.net.URI; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; +import java.util.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.StorageType; import org.junit.Test; + +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.*; import static org.mockito.Mockito.*; @@ -34,19 +36,69 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode.DataNodeDiskChecker; public class TestDataDirs { - @Test (timeout = 10000) - public void testGetDataDirsFromURIs() throws Throwable { + @Test (timeout = 30000) + public void testDataDirParsing() throws Throwable { + Configuration conf = new Configuration(); + List locations; + File dir0 = new File("/dir0"); + File dir1 = new File("/dir1"); + File dir2 = new File("/dir2"); + File dir3 = new File("/dir3"); + + // Verify that a valid string is correctly parsed, and that storage + // type is not case-sensitive + String locations1 = "[disk]/dir0,[DISK]/dir1,[sSd]/dir2,[disK]/dir3"; + conf.set(DFS_DATANODE_DATA_DIR_KEY, locations1); + locations = DataNode.getStorageLocations(conf); + assertThat(locations.size(), is(4)); + assertThat(locations.get(0).getStorageType(), is(StorageType.DISK)); + assertThat(locations.get(0).getUri(), is(dir0.toURI())); + assertThat(locations.get(1).getStorageType(), is(StorageType.DISK)); + assertThat(locations.get(1).getUri(), is(dir1.toURI())); + assertThat(locations.get(2).getStorageType(), is(StorageType.SSD)); + assertThat(locations.get(2).getUri(), is(dir2.toURI())); + assertThat(locations.get(3).getStorageType(), is(StorageType.DISK)); + assertThat(locations.get(3).getUri(), is(dir3.toURI())); + + // Verify that an unrecognized storage type result in an exception. + String locations2 = "[BadMediaType]/dir0,[ssd]/dir1,[disk]/dir2"; + conf.set(DFS_DATANODE_DATA_DIR_KEY, locations2); + try { + locations = DataNode.getStorageLocations(conf); + fail(); + } catch(IllegalArgumentException iae) { + DataNode.LOG.info("The exception is expected.", iae); + } + + // Assert that a string with no storage type specified is + // correctly parsed and the default storage type is picked up. + String locations3 = "/dir0,/dir1"; + conf.set(DFS_DATANODE_DATA_DIR_KEY, locations3); + locations = DataNode.getStorageLocations(conf); + assertThat(locations.size(), is(2)); + assertThat(locations.get(0).getStorageType(), is(StorageType.DISK)); + assertThat(locations.get(0).getUri(), is(dir0.toURI())); + assertThat(locations.get(1).getStorageType(), is(StorageType.DISK)); + assertThat(locations.get(1).getUri(), is(dir1.toURI())); + } + + @Test (timeout = 30000) + public void testDataDirValidation() throws Throwable { DataNodeDiskChecker diskChecker = mock(DataNodeDiskChecker.class); doThrow(new IOException()).doThrow(new IOException()).doNothing() .when(diskChecker).checkDir(any(LocalFileSystem.class), any(Path.class)); LocalFileSystem fs = mock(LocalFileSystem.class); - Collection uris = Arrays.asList(new URI("file:/p1/"), - new URI("file:/p2/"), new URI("file:/p3/")); + AbstractList locations = new ArrayList(); - List dirs = DataNode.getDataDirsFromURIs(uris, fs, diskChecker); - assertEquals("number of valid data dirs", 1, dirs.size()); - String validDir = dirs.iterator().next().getPath(); - assertEquals("p3 should be valid", new File("/p3").getPath(), validDir); + locations.add(StorageLocation.parse("file:/p1/")); + locations.add(StorageLocation.parse("file:/p2/")); + locations.add(StorageLocation.parse("file:/p3/")); + + List checkedLocations = + DataNode.checkStorageLocations(locations, fs, diskChecker); + assertEquals("number of valid data dirs", 1, checkedLocations.size()); + String validDir = checkedLocations.iterator().next().getFile().getPath(); + assertThat("p3 should be valid", new File("/p3/").getPath(), is(validDir)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java index 50272f1dda2..8ce8569df41 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java @@ -163,7 +163,7 @@ public class TestDataNodeMultipleRegistrations { for (BPOfferService bpos : dn.getAllBpOs()) { LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration + "; sid=" - + bpos.bpRegistration.getStorageID() + "; nna=" + + + bpos.bpRegistration.getDatanodeUuid() + "; nna=" + getNNSocketAddress(bpos)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java index 75482ae4426..646d33d13b8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java @@ -42,11 +42,13 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.net.TcpPeerServer; import org.apache.hadoop.hdfs.protocol.Block; +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.LocatedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; @@ -151,13 +153,23 @@ public class TestDataNodeVolumeFailure { DataNode dn = cluster.getDataNodes().get(1); //corresponds to dir data3 String bpid = cluster.getNamesystem().getBlockPoolId(); DatanodeRegistration dnR = dn.getDNRegistrationForBP(bpid); - final StorageBlockReport[] report = { - new StorageBlockReport( - new DatanodeStorage(dnR.getStorageID()), - DataNodeTestUtils.getFSDataset(dn).getBlockReport(bpid - ).getBlockListAsLongs()) - }; - cluster.getNameNodeRpc().blockReport(dnR, bpid, report); + + Map perVolumeBlockLists = + dn.getFSDataset().getBlockReports(bpid); + + // Send block report + StorageBlockReport[] reports = + new StorageBlockReport[perVolumeBlockLists.size()]; + + int reportIndex = 0; + for(Map.Entry kvPair : perVolumeBlockLists.entrySet()) { + DatanodeStorage dnStorage = kvPair.getKey(); + BlockListAsLongs blockList = kvPair.getValue(); + reports[reportIndex++] = + new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs()); + } + + cluster.getNameNodeRpc().blockReport(dnR, bpid, reports); // verify number of blocks and files... verify(filename, filesize); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java index c5decf2eaa6..f5b535d3943 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; @@ -406,6 +407,16 @@ public class TestDirectoryScanner { public File getFinalizedDir(String bpid) throws IOException { return new File("/base/current/" + bpid + "/finalized"); } + + @Override + public StorageType getStorageType() { + return StorageType.DEFAULT; + } + + @Override + public String getStorageID() { + return ""; + } } private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi(); @@ -436,7 +447,7 @@ public class TestDirectoryScanner { void testScanInfoObject(long blockId) throws Exception { DirectoryScanner.ScanInfo scanInfo = - new DirectoryScanner.ScanInfo(blockId); + new DirectoryScanner.ScanInfo(blockId, null, null, null); assertEquals(blockId, scanInfo.getBlockId()); assertNull(scanInfo.getBlockFile()); assertNull(scanInfo.getMetaFile()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java index e8630091f65..d03e5ea0252 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java @@ -311,7 +311,7 @@ public class TestSimulatedFSDataset { } private SimulatedFSDataset getSimulatedFSDataset() { - SimulatedFSDataset fsdataset = new SimulatedFSDataset(null, null, conf); + SimulatedFSDataset fsdataset = new SimulatedFSDataset(null, conf); fsdataset.addBlockPool(bpid, conf); return fsdataset; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java index 69cd684b9ec..47d97a887d0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java @@ -17,6 +17,14 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.impl.Log4JLogger; @@ -25,28 +33,40 @@ import org.apache.hadoop.fs.CreateFlag; 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.*; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; +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.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataStorage; -import org.apache.hadoop.hdfs.server.protocol.*; +import org.apache.hadoop.hdfs.server.protocol.BlockCommand; +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.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; +import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.net.DNS; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.security.Groups; -import org.apache.hadoop.util.*; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.Time; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.util.VersionInfo; import org.apache.log4j.Level; import org.apache.log4j.LogManager; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.EnumSet; -import java.util.List; - /** * Main class for a series of name-node benchmarks. * @@ -909,17 +929,16 @@ public class NNThroughputBenchmark implements Tool { dnRegistration = new DatanodeRegistration( new DatanodeID(DNS.getDefaultIP("default"), DNS.getDefaultHost("default", "default"), - "", getNodePort(dnIdx), + DataNode.generateUuid(), getNodePort(dnIdx), DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT, DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT), - new DataStorage(nsInfo, ""), + new DataStorage(nsInfo), new ExportedBlockKeys(), VersionInfo.getVersion()); - DataNode.setNewStorageID(dnRegistration); // register datanode dnRegistration = nameNodeProto.registerDatanode(dnRegistration); //first block reports - storage = new DatanodeStorage(dnRegistration.getStorageID()); + storage = new DatanodeStorage(dnRegistration.getDatanodeUuid()); final StorageBlockReport[] reports = { new StorageBlockReport(storage, new BlockListAsLongs(null, null).getBlockListAsLongs()) @@ -935,7 +954,7 @@ public class NNThroughputBenchmark implements Tool { void sendHeartbeat() throws IOException { // register datanode // TODO:FEDERATION currently a single block pool is supported - StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(), + StorageReport[] rep = { new StorageReport(dnRegistration.getDatanodeUuid(), false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) }; DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration, rep, 0, 0, 0).getCommands(); @@ -982,7 +1001,7 @@ public class NNThroughputBenchmark implements Tool { @SuppressWarnings("unused") // keep it for future blockReceived benchmark int replicateBlocks() throws IOException { // register datanode - StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(), + StorageReport[] rep = { new StorageReport(dnRegistration.getDatanodeUuid(), false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) }; DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration, rep, 0, 0, 0).getCommands(); @@ -1012,14 +1031,14 @@ public class NNThroughputBenchmark implements Tool { DatanodeInfo dnInfo = blockTargets[t]; DatanodeRegistration receivedDNReg; receivedDNReg = new DatanodeRegistration(dnInfo, - new DataStorage(nsInfo, dnInfo.getStorageID()), + new DataStorage(nsInfo), new ExportedBlockKeys(), VersionInfo.getVersion()); ReceivedDeletedBlockInfo[] rdBlocks = { new ReceivedDeletedBlockInfo( blocks[i], ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null) }; StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks( - receivedDNReg.getStorageID(), rdBlocks) }; + receivedDNReg.getDatanodeUuid(), rdBlocks) }; nameNodeProto.blockReceivedAndDeleted(receivedDNReg, nameNode .getNamesystem().getBlockPoolId(), report); } @@ -1142,7 +1161,7 @@ public class NNThroughputBenchmark implements Tool { loc.getBlock().getLocalBlock(), ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null) }; StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks( - datanodes[dnIdx].dnRegistration.getStorageID(), rdBlocks) }; + datanodes[dnIdx].dnRegistration.getDatanodeUuid(), rdBlocks) }; nameNodeProto.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration, loc .getBlock().getBlockPoolId(), report); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java index 64113dc91a2..0ebf929c2bc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp; @@ -111,8 +112,9 @@ public class NameNodeAdapter { public static HeartbeatResponse sendHeartBeat(DatanodeRegistration nodeReg, DatanodeDescriptor dd, FSNamesystem namesystem) throws IOException { - return namesystem.handleHeartbeat(nodeReg, dd.getCapacity(), - dd.getDfsUsed(), dd.getRemaining(), dd.getBlockPoolUsed(), 0, 0, 0); + return namesystem.handleHeartbeat(nodeReg, + BlockManagerTestUtil.getStorageReportsForDatanode(dd), + 0, 0, 0); } public static boolean setReplication(final FSNamesystem ns, @@ -242,3 +244,4 @@ public class NameNodeAdapter { return NNStorage.getInProgressEditsFile(sd, startTxId); } } + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java index 3114c824cea..08c44c23edb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.net.Node; @@ -99,13 +100,13 @@ public class TestAddBlockRetry { bmField.setAccessible(true); bmField.set(ns, spyBM); - doAnswer(new Answer() { + doAnswer(new Answer() { @Override - public DatanodeDescriptor[] answer(InvocationOnMock invocation) + public DatanodeStorageInfo[] answer(InvocationOnMock invocation) throws Throwable { LOG.info("chooseTarget for " + src); - DatanodeDescriptor[] ret = - (DatanodeDescriptor[]) invocation.callRealMethod(); + DatanodeStorageInfo[] ret = + (DatanodeStorageInfo[]) invocation.callRealMethod(); count++; if(count == 1) { // run second addBlock() LOG.info("Starting second addBlock for " + src); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java index 83a58652910..256a10c486b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.junit.Test; @@ -48,7 +48,7 @@ public class TestCommitBlockSynchronization { throws IOException { Configuration conf = new Configuration(); FSImage image = new FSImage(conf); - DatanodeDescriptor[] targets = new DatanodeDescriptor[0]; + final DatanodeStorageInfo[] targets = {}; FSNamesystem namesystem = new FSNamesystem(conf, image); FSNamesystem namesystemSpy = spy(namesystem); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java index d78198ab402..5461a09c260 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java @@ -104,11 +104,11 @@ public class TestDeadDatanode { DatanodeRegistration reg = DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId); - waitForDatanodeState(reg.getStorageID(), true, 20000); + waitForDatanodeState(reg.getDatanodeUuid(), true, 20000); // Shutdown and wait for datanode to be marked dead dn.shutdown(); - waitForDatanodeState(reg.getStorageID(), false, 20000); + waitForDatanodeState(reg.getDatanodeUuid(), false, 20000); DatanodeProtocol dnp = cluster.getNameNodeRpc(); @@ -117,7 +117,7 @@ public class TestDeadDatanode { ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null) }; StorageReceivedDeletedBlocks[] storageBlocks = { - new StorageReceivedDeletedBlocks(reg.getStorageID(), blocks) }; + new StorageReceivedDeletedBlocks(reg.getDatanodeUuid(), blocks) }; // Ensure blockReceived call from dead datanode is rejected with IOException try { @@ -129,7 +129,7 @@ public class TestDeadDatanode { // Ensure blockReport from dead datanode is rejected with IOException StorageBlockReport[] report = { new StorageBlockReport( - new DatanodeStorage(reg.getStorageID()), + new DatanodeStorage(reg.getDatanodeUuid()), new long[] { 0L, 0L, 0L }) }; try { dnp.blockReport(reg, poolId, report); @@ -140,7 +140,7 @@ public class TestDeadDatanode { // Ensure heartbeat from dead datanode is rejected with a command // that asks datanode to register again - StorageReport[] rep = { new StorageReport(reg.getStorageID(), false, 0, 0, + StorageReport[] rep = { new StorageReport(reg.getDatanodeUuid(), false, 0, 0, 0, 0) }; DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0, 0, 0).getCommands(); assertEquals(1, cmd.length); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java index 0995e006ce3..d635df0c658 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java @@ -55,7 +55,7 @@ public class TestListCorruptFileBlocks { static Log LOG = NameNode.stateChangeLog; /** check if nn.getCorruptFiles() returns a file that has corrupted blocks */ - @Test + @Test (timeout=300000) public void testListCorruptFilesCorruptedBlock() throws Exception { MiniDFSCluster cluster = null; Random random = new Random(); @@ -131,7 +131,7 @@ public class TestListCorruptFileBlocks { /** * Check that listCorruptFileBlocks works while the namenode is still in safemode. */ - @Test + @Test (timeout=300000) public void testListCorruptFileBlocksInSafeMode() throws Exception { MiniDFSCluster cluster = null; Random random = new Random(); @@ -262,7 +262,7 @@ public class TestListCorruptFileBlocks { } // deliberately remove blocks from a file and validate the list-corrupt-file-blocks API - @Test + @Test (timeout=300000) public void testlistCorruptFileBlocks() throws Exception { Configuration conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000); @@ -372,7 +372,7 @@ public class TestListCorruptFileBlocks { /** * test listCorruptFileBlocks in DistributedFileSystem */ - @Test + @Test (timeout=300000) public void testlistCorruptFileBlocksDFS() throws Exception { Configuration conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000); @@ -446,7 +446,7 @@ public class TestListCorruptFileBlocks { * Also, test that DFS.listCorruptFileBlocks can make multiple successive * calls. */ - @Test + @Test (timeout=300000) public void testMaxCorruptFiles() throws Exception { MiniDFSCluster cluster = null; try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java index ddb7c0fa692..65032f29b3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java @@ -316,10 +316,11 @@ public class TestNamenodeRetryCache { ExtendedBlock oldBlock = new ExtendedBlock(); ExtendedBlock newBlock = new ExtendedBlock(); DatanodeID[] newNodes = new DatanodeID[2]; + String[] newStorages = new String[2]; newCall(); try { - ns0.updatePipeline("testClient", oldBlock, newBlock, newNodes); + ns0.updatePipeline("testClient", oldBlock, newBlock, newNodes, newStorages); fail("Expect StandbyException from the updatePipeline call"); } catch (StandbyException e) { // expected, since in the beginning both nn are in standby state @@ -329,7 +330,7 @@ public class TestNamenodeRetryCache { cluster.transitionToActive(0); try { - ns0.updatePipeline("testClient", oldBlock, newBlock, newNodes); + ns0.updatePipeline("testClient", oldBlock, newBlock, newNodes, newStorages); } catch (IOException e) { // ignore call should not hang. } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java index 3c8b91be8d2..7c87ab0d664 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; @@ -523,16 +524,17 @@ public class TestPipelinesFailover { (BlockInfoUnderConstruction)storedBlock; // We expect that the replica with the most recent heart beat will be // the one to be in charge of the synchronization / recovery protocol. - DatanodeDescriptor[] datanodes = ucBlock.getExpectedLocations(); - DatanodeDescriptor expectedPrimary = datanodes[0]; - long mostRecentLastUpdate = expectedPrimary.getLastUpdate(); - for (int i = 1; i < datanodes.length; i++) { - if (datanodes[i].getLastUpdate() > mostRecentLastUpdate) { - expectedPrimary = datanodes[i]; - mostRecentLastUpdate = expectedPrimary.getLastUpdate(); + final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations(); + DatanodeStorageInfo expectedPrimary = storages[0]; + long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor().getLastUpdate(); + for (int i = 1; i < storages.length; i++) { + final long lastUpdate = storages[i].getDatanodeDescriptor().getLastUpdate(); + if (lastUpdate > mostRecentLastUpdate) { + expectedPrimary = storages[i]; + mostRecentLastUpdate = lastUpdate; } } - return expectedPrimary; + return expectedPrimary.getDatanodeDescriptor(); } private DistributedFileSystem createFsAsOtherUser( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java index 82deab59386..e26790311b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java @@ -707,9 +707,10 @@ public class TestRetryCacheWithHA { DatanodeInfo[] newNodes = new DatanodeInfo[2]; newNodes[0] = nodes[0]; newNodes[1] = nodes[1]; + String[] storageIDs = {"s0", "s1"}; client.getNamenode().updatePipeline(client.getClientName(), oldBlock, - newBlock, newNodes); + newBlock, newNodes, storageIDs); out.close(); } @@ -719,10 +720,10 @@ public class TestRetryCacheWithHA { .getNamesystem(0).getFSDirectory().getINode4Write(file).asFile(); BlockInfoUnderConstruction blkUC = (BlockInfoUnderConstruction) (fileNode.getBlocks())[1]; - int datanodeNum = blkUC.getExpectedLocations().length; + int datanodeNum = blkUC.getExpectedStorageLocations().length; for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) { Thread.sleep(1000); - datanodeNum = blkUC.getExpectedLocations().length; + datanodeNum = blkUC.getExpectedStorageLocations().length; } return datanodeNum == 2; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java index c00f369f9b4..d1ee008d69f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java @@ -238,7 +238,7 @@ public class TestNameNodeMetrics { cluster.getNamesystem().writeLock(); try { bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0], - "TEST"); + "STORAGE_ID", "TEST"); } finally { cluster.getNamesystem().writeUnlock(); } @@ -281,7 +281,7 @@ public class TestNameNodeMetrics { cluster.getNamesystem().writeLock(); try { bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0], - "TEST"); + "STORAGE_ID", "TEST"); } finally { cluster.getNamesystem().writeUnlock(); } @@ -443,7 +443,8 @@ public class TestNameNodeMetrics { // We have one sync when the cluster starts up, just opening the journal assertCounter("SyncsNumOps", 1L, rb); // Each datanode reports in when the cluster comes up - assertCounter("BlockReportNumOps", (long)DATANODE_COUNT, rb); + assertCounter("BlockReportNumOps", + (long)DATANODE_COUNT*MiniDFSCluster.DIRS_PER_DATANODE, rb); // Sleep for an interval+slop to let the percentiles rollover Thread.sleep((PERCENTILES_INTERVAL+1)*1000); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored index 111cec0bef7..3c7fcbbf980 100644 Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored differ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml index dbb3498d46d..6b9160fc4b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml @@ -1,6 +1,6 @@ - -47 + -49 OP_START_LOG_SEGMENT @@ -13,8 +13,8 @@ 2 1 - 1375509063810 - 4d47710649039b98 + 1388803909637 + 02b5879ef37263c7 @@ -24,8 +24,8 @@ 3 2 - 1375509063812 - 38cbb1d8fd90fcb2 + 1388803909659 + ae5f64e833bca20c @@ -37,17 +37,17 @@ 16386 /file_create_u\0001;F431 1 - 1374817864805 - 1374817864805 + 1388112711695 + 1388112711695 512 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 127.0.0.1 - jing + aagarwal supergroup 420 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 8 @@ -59,13 +59,13 @@ 0 /file_create_u\0001;F431 1 - 1374817864816 - 1374817864805 + 1388112711779 + 1388112711695 512 - jing + aagarwal supergroup 420 @@ -78,8 +78,8 @@ 0 /file_create_u\0001;F431 /file_moved - 1374817864818 - 5245793a-984b-4264-8d7c-7890775547a0 + 1388112711794 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 10 @@ -89,8 +89,8 @@ 7 0 /file_moved - 1374817864822 - 5245793a-984b-4264-8d7c-7890775547a0 + 1388112711831 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 11 @@ -101,9 +101,9 @@ 0 16387 /directory_mkdir - 1374817864825 + 1388112711853 - jing + aagarwal supergroup 493 @@ -136,7 +136,7 @@ 12 /directory_mkdir snapshot1 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 16 @@ -147,7 +147,7 @@ /directory_mkdir snapshot1 snapshot2 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 17 @@ -157,7 +157,7 @@ 14 /directory_mkdir snapshot2 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 18 @@ -169,17 +169,17 @@ 16388 /file_create_u\0001;F431 1 - 1374817864846 - 1374817864846 + 1388112711885 + 1388112711885 512 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 127.0.0.1 - jing + aagarwal supergroup 420 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 19 @@ -191,13 +191,13 @@ 0 /file_create_u\0001;F431 1 - 1374817864848 - 1374817864846 + 1388112711899 + 1388112711885 512 - jing + aagarwal supergroup 420 @@ -253,9 +253,9 @@ 0 /file_create_u\0001;F431 /file_moved - 1374817864860 + 1388112711919 NONE - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 26 @@ -267,17 +267,17 @@ 16389 /file_concat_target 1 - 1374817864864 - 1374817864864 + 1388112711932 + 1388112711932 512 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 127.0.0.1 - jing + aagarwal supergroup 420 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 28 @@ -388,8 +388,8 @@ 0 /file_concat_target 1 - 1374817864927 - 1374817864864 + 1388112712188 + 1388112711932 512 @@ -409,7 +409,7 @@ 1003 - jing + aagarwal supergroup 420 @@ -423,17 +423,17 @@ 16390 /file_concat_0 1 - 1374817864929 - 1374817864929 + 1388112712191 + 1388112712191 512 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 127.0.0.1 - jing + aagarwal supergroup 420 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 41 @@ -544,8 +544,8 @@ 0 /file_concat_0 1 - 1374817864947 - 1374817864929 + 1388112712212 + 1388112712191 512 @@ -565,7 +565,7 @@ 1006 - jing + aagarwal supergroup 420 @@ -579,17 +579,17 @@ 16391 /file_concat_1 1 - 1374817864950 - 1374817864950 + 1388112712215 + 1388112712215 512 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 127.0.0.1 - jing + aagarwal supergroup 420 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 53 @@ -700,8 +700,8 @@ 0 /file_concat_1 1 - 1374817864966 - 1374817864950 + 1388112712238 + 1388112712215 512 @@ -721,7 +721,7 @@ 1009 - jing + aagarwal supergroup 420 @@ -733,12 +733,12 @@ 56 0 /file_concat_target - 1374817864967 + 1388112712241 /file_concat_0 /file_concat_1 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 64 @@ -750,14 +750,14 @@ 16392 /file_symlink /file_concat_target - 1374817864971 - 1374817864971 + 1388112712273 + 1388112712273 - jing + aagarwal supergroup 511 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 65 @@ -768,14 +768,14 @@ HDFS_DELEGATION_TOKEN 1 - jing + aagarwal JobTracker - 1374817864974 - 1375422664974 + 1388112712276 + 1388717512276 2 - 1374904264974 + 1388199112276 @@ -785,14 +785,14 @@ HDFS_DELEGATION_TOKEN 1 - jing + aagarwal JobTracker - 1374817864974 - 1375422664974 + 1388112712276 + 1388717512276 2 - 1374904265012 + 1388199112421 @@ -802,11 +802,11 @@ HDFS_DELEGATION_TOKEN 1 - jing + aagarwal JobTracker - 1374817864974 - 1375422664974 + 1388112712276 + 1388717512276 2 @@ -819,17 +819,17 @@ 16393 /hard-lease-recovery-test 1 - 1374817865017 - 1374817865017 + 1388112712441 + 1388112712441 512 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 127.0.0.1 - jing + aagarwal supergroup 420 - 5245793a-984b-4264-8d7c-7890775547a0 + 90461de5-e4ed-45f8-bee4-3b0e4c7979bd 69 @@ -868,7 +868,7 @@ /hard-lease-recovery-test 1073741834 - 0 + 11 1010 @@ -886,7 +886,23 @@ OP_REASSIGN_LEASE 67 - DFSClient_NONMAPREDUCE_-1676409172_1 + DFSClient_NONMAPREDUCE_1163912231_1 + /hard-lease-recovery-test + HDFS_NameNode + + + + OP_SET_GENSTAMP_V2 + + 68 + 1012 + + + + OP_REASSIGN_LEASE + + 69 + HDFS_NameNode /hard-lease-recovery-test HDFS_NameNode @@ -894,23 +910,23 @@ OP_CLOSE - 68 + 70 0 0 /hard-lease-recovery-test 1 - 1374817867688 - 1374817865017 + 1388112717425 + 1388112712441 512 1073741834 11 - 1011 + 1012 - jing + aagarwal supergroup 420 @@ -919,7 +935,7 @@ OP_END_LOG_SEGMENT - 69 + 71