Return-Path: X-Original-To: apmail-hadoop-common-commits-archive@www.apache.org Delivered-To: apmail-hadoop-common-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1A1B817329 for ; Tue, 2 Jun 2015 23:54:55 +0000 (UTC) Received: (qmail 88305 invoked by uid 500); 2 Jun 2015 23:47:33 -0000 Delivered-To: apmail-hadoop-common-commits-archive@hadoop.apache.org Received: (qmail 87675 invoked by uid 500); 2 Jun 2015 23:47:33 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: common-dev@hadoop.apache.org Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 87641 invoked by uid 99); 2 Jun 2015 23:47:33 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 02 Jun 2015 23:47:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 46813DFAF3; Tue, 2 Jun 2015 23:47:33 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: zjshen@apache.org To: common-commits@hadoop.apache.org Date: Tue, 02 Jun 2015 23:47:36 -0000 Message-Id: In-Reply-To: <098e82e7de634dada8047c51499cc077@git.apache.org> References: <098e82e7de634dada8047c51499cc077@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [04/50] [abbrv] hadoop git commit: HDFS-8482. Rename BlockInfoContiguous to BlockInfo. Contributed by Zhe Zhang. http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java index 476ff36..9acd81f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java @@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LayoutVersion; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; @@ -942,7 +942,7 @@ public class FSEditLogLoader { */ private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file) throws IOException { - BlockInfoContiguous[] oldBlocks = file.getBlocks(); + BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); Block newBlock= op.getLastBlock(); @@ -968,7 +968,7 @@ public class FSEditLogLoader { Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0); } // add the new block - BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction( + BlockInfo newBI = new BlockInfoContiguousUnderConstruction( newBlock, file.getPreferredBlockReplication()); fsNamesys.getBlockManager().addBlockCollection(newBI, file); file.addBlock(newBI); @@ -982,7 +982,7 @@ public class FSEditLogLoader { private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, INodesInPath iip, INodeFile file) throws IOException { // Update its block list - BlockInfoContiguous[] oldBlocks = file.getBlocks(); + BlockInfo[] oldBlocks = file.getBlocks(); Block[] newBlocks = op.getBlocks(); String path = op.getPath(); @@ -991,7 +991,7 @@ public class FSEditLogLoader { // First, update blocks in common for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) { - BlockInfoContiguous oldBlock = oldBlocks[i]; + BlockInfo oldBlock = oldBlocks[i]; Block newBlock = newBlocks[i]; boolean isLastBlock = i == newBlocks.length - 1; @@ -1043,7 +1043,7 @@ public class FSEditLogLoader { // We're adding blocks for (int i = oldBlocks.length; i < newBlocks.length; i++) { Block newBlock = newBlocks[i]; - BlockInfoContiguous newBI; + BlockInfo newBI; if (!op.shouldCompleteLastBlock()) { // TODO: shouldn't this only be true for the last block? // what about an old-version fsync() where fsync isn't called @@ -1055,7 +1055,7 @@ public class FSEditLogLoader { // is only executed when loading edits written by prior // versions of Hadoop. Current versions always log // OP_ADD operations as each block is allocated. - newBI = new BlockInfoContiguous(newBlock, + newBI = new BlockInfo(newBlock, file.getPreferredBlockReplication()); } fsNamesys.getBlockManager().addBlockCollection(newBI, file); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java index ec2babd..a46f2b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java @@ -52,7 +52,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LayoutFlags; import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -686,7 +686,7 @@ public class FSImageFormat { public void updateBlocksMap(INodeFile file) { // Add file->block mapping - final BlockInfoContiguous[] blocks = file.getBlocks(); + final BlockInfo[] blocks = file.getBlocks(); if (blocks != null) { final BlockManager bm = namesystem.getBlockManager(); for (int i = 0; i < blocks.length; i++) { @@ -753,9 +753,9 @@ public class FSImageFormat { // file // read blocks - BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks]; + BlockInfo[] blocks = new BlockInfo[numBlocks]; for (int j = 0; j < numBlocks; j++) { - blocks[j] = new BlockInfoContiguous(replication); + blocks[j] = new BlockInfo(replication); blocks[j].readFields(in); } @@ -775,7 +775,7 @@ public class FSImageFormat { clientMachine = FSImageSerialization.readString(in); // convert the last block to BlockUC if (blocks.length > 0) { - BlockInfoContiguous lastBlk = blocks[blocks.length - 1]; + BlockInfo lastBlk = blocks[blocks.length - 1]; blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction( lastBlk, replication); } @@ -958,9 +958,9 @@ public class FSImageFormat { FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature(); oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine()); if (oldnode.numBlocks() > 0) { - BlockInfoContiguous ucBlock = cons.getLastBlock(); + BlockInfo ucBlock = cons.getLastBlock(); // we do not replace the inode, just replace the last block of oldnode - BlockInfoContiguous info = namesystem.getBlockManager().addBlockCollection( + BlockInfo info = namesystem.getBlockManager().addBlockCollection( ucBlock, oldnode); oldnode.setBlock(oldnode.numBlocks() - 1, info); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java index d966c69..29713a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -43,7 +42,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; @@ -211,7 +210,7 @@ public final class FSImageFormatPBINode { public static void updateBlocksMap(INodeFile file, BlockManager bm) { // Add file->block mapping - final BlockInfoContiguous[] blocks = file.getBlocks(); + final BlockInfo[] blocks = file.getBlocks(); if (blocks != null) { for (int i = 0; i < blocks.length; i++) { file.setBlock(i, bm.addBlockCollection(blocks[i], file)); @@ -324,9 +323,9 @@ public final class FSImageFormatPBINode { short replication = (short) f.getReplication(); LoaderContext state = parent.getLoaderContext(); - BlockInfoContiguous[] blocks = new BlockInfoContiguous[bp.size()]; + BlockInfo[] blocks = new BlockInfo[bp.size()]; for (int i = 0, e = bp.size(); i < e; ++i) { - blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication); + blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication); } final PermissionStatus permissions = loadPermission(f.getPermission(), parent.getLoaderContext().getStringTable()); @@ -352,7 +351,7 @@ public final class FSImageFormatPBINode { INodeSection.FileUnderConstructionFeature uc = f.getFileUC(); file.toUnderConstruction(uc.getClientName(), uc.getClientMachine()); if (blocks.length > 0) { - BlockInfoContiguous lastBlk = file.getLastBlock(); + BlockInfo lastBlk = file.getLastBlock(); // replace the last block of file file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction( lastBlk, replication)); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java index 1888d87..d790b8f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.LayoutVersion; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; @@ -126,12 +126,12 @@ public class FSImageSerialization { long preferredBlockSize = in.readLong(); int numBlocks = in.readInt(); - BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks]; + BlockInfo[] blocks = new BlockInfo[numBlocks]; Block blk = new Block(); int i = 0; for (; i < numBlocks-1; i++) { blk.readFields(in); - blocks[i] = new BlockInfoContiguous(blk, blockReplication); + blocks[i] = new BlockInfo(blk, blockReplication); } // last block is UNDER_CONSTRUCTION if(numBlocks > 0) { http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java ---------------------------------------------------------------------- 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 bfd6eba..798f8d5 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 @@ -203,7 +203,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -2016,7 +2016,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, } // Check if the file is already being truncated with the same length - final BlockInfoContiguous last = file.getLastBlock(); + final BlockInfo last = file.getLastBlock(); if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { final Block truncateBlock = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock(); @@ -2090,7 +2090,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, leaseManager.addLease( file.getFileUnderConstructionFeature().getClientName(), file.getId()); boolean shouldRecoverNow = (newBlock == null); - BlockInfoContiguous oldBlock = file.getLastBlock(); + BlockInfo oldBlock = file.getLastBlock(); boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock); if(newBlock == null) { newBlock = (shouldCopyOnTruncate) ? createNewBlock() : @@ -2141,7 +2141,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, * Defines if a replica needs to be copied on truncate or * can be truncated in place. */ - boolean shouldCopyOnTruncate(INodeFile file, BlockInfoContiguous blk) { + boolean shouldCopyOnTruncate(INodeFile file, BlockInfo blk) { if(!isUpgradeFinalized()) { return true; } @@ -2505,7 +2505,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, src, holder, clientMachine, false); - final BlockInfoContiguous lastBlock = myFile.getLastBlock(); + final BlockInfo lastBlock = myFile.getLastBlock(); // Check that the block has at least minimum replication. if(lastBlock != null && lastBlock.isComplete() && !getBlockManager().isSufficientlyReplicated(lastBlock)) { @@ -2561,7 +2561,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, } } } else { - BlockInfoContiguous lastBlock = file.getLastBlock(); + BlockInfo lastBlock = file.getLastBlock(); if (lastBlock != null) { ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock); ret = new LocatedBlock(blk, new DatanodeInfo[0]); @@ -2603,7 +2603,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, /** Compute quota change for converting a complete block to a UC block */ private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) { final QuotaCounts delta = new QuotaCounts.Builder().build(); - final BlockInfoContiguous lastBlock = file.getLastBlock(); + final BlockInfo lastBlock = file.getLastBlock(); if (lastBlock != null) { final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes(); final short repl = file.getPreferredBlockReplication(); @@ -2740,7 +2740,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, op.getExceptionMessage(src, holder, clientMachine, "lease recovery is in progress. Try again later.")); } else { - final BlockInfoContiguous lastBlock = file.getLastBlock(); + final BlockInfo lastBlock = file.getLastBlock(); if (lastBlock != null && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { throw new RecoveryInProgressException( @@ -3065,10 +3065,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, .getBlocks()); } else { // check the penultimate block of this file - BlockInfoContiguous b = v.getPenultimateBlock(); + BlockInfo b = v.getPenultimateBlock(); return b == null || blockManager.checkBlocksProperlyReplicated( - src, new BlockInfoContiguous[] { b }); + src, new BlockInfo[] { b }); } } @@ -3242,7 +3242,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, for (Block b : blocks.getToDeleteList()) { if (trackBlockCounts) { - BlockInfoContiguous bi = getStoredBlock(b); + BlockInfo bi = getStoredBlock(b); if (bi.isComplete()) { numRemovedComplete++; if (bi.numNodes() >= blockManager.minReplication) { @@ -3466,10 +3466,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, final INodeFile pendingFile = iip.getLastINode().asFile(); int nrBlocks = pendingFile.numBlocks(); - BlockInfoContiguous[] blocks = pendingFile.getBlocks(); + BlockInfo[] blocks = pendingFile.getBlocks(); int nrCompleteBlocks; - BlockInfoContiguous curBlock = null; + BlockInfo curBlock = null; for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) { curBlock = blocks[nrCompleteBlocks]; if(!curBlock.isComplete()) @@ -3504,9 +3504,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, // The last block is not COMPLETE, and // that the penultimate block if exists is either COMPLETE or COMMITTED - final BlockInfoContiguous lastBlock = pendingFile.getLastBlock(); + final BlockInfo lastBlock = pendingFile.getLastBlock(); BlockUCState lastBlockState = lastBlock.getBlockUCState(); - BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock(); + BlockInfo penultimateBlock = pendingFile.getPenultimateBlock(); // If penultimate block doesn't exist then its minReplication is met boolean penultimateBlockMinReplication = penultimateBlock == null ? true : @@ -3650,7 +3650,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, } @VisibleForTesting - BlockInfoContiguous getStoredBlock(Block block) { + BlockInfo getStoredBlock(Block block) { return blockManager.getStoredBlock(block); } @@ -3710,7 +3710,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, checkNameNodeSafeMode( "Cannot commitBlockSynchronization while in safe mode"); - final BlockInfoContiguous storedBlock = getStoredBlock( + final BlockInfo storedBlock = getStoredBlock( ExtendedBlock.getLocalBlock(oldBlock)); if (storedBlock == null) { if (deleteblock) { @@ -3871,7 +3871,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, * @throws IOException on error */ @VisibleForTesting - String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock) + String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock) throws IOException { final INodesInPath iip = INodesInPath.fromINode(pendingFile); final String src = iip.getPath(); @@ -4162,7 +4162,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, while (it.hasNext()) { Block b = it.next(); - BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b); + BlockInfo blockInfo = blockManager.getStoredBlock(b); if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) { filesToDelete.add(blockInfo.getBlockCollection()); @@ -5105,7 +5105,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, SafeModeInfo safeMode = this.safeMode; if (safeMode == null) // mostly true return; - BlockInfoContiguous storedBlock = getStoredBlock(b); + BlockInfo storedBlock = getStoredBlock(b); if (storedBlock.isComplete()) { safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas()); } @@ -5665,7 +5665,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, + "access token for block " + block); // check stored block state - BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block)); + BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block)); if (storedBlock == null || storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) { throw new IOException(block + http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java index 1ebdde6..d07ae1f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode; import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; @@ -58,7 +58,7 @@ public class FileUnderConstructionFeature implements INode.Feature { */ void updateLengthOfLastBlock(INodeFile f, long lastBlockLength) throws IOException { - BlockInfoContiguous lastBlock = f.getLastBlock(); + BlockInfo lastBlock = f.getLastBlock(); assert (lastBlock != null) : "The last block for path " + f.getFullPathName() + " is null when updating its length"; assert (lastBlock instanceof BlockInfoContiguousUnderConstruction) @@ -74,7 +74,7 @@ public class FileUnderConstructionFeature implements INode.Feature { */ void cleanZeroSizeBlock(final INodeFile f, final BlocksMapUpdateInfo collectedBlocks) { - final BlockInfoContiguous[] blocks = f.getBlocks(); + final BlockInfo[] blocks = f.getBlocks(); if (blocks != null && blocks.length > 0 && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) { BlockInfoContiguousUnderConstruction lastUC = http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java index a6ff6fb..294323c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -121,17 +121,17 @@ public class INodeFile extends INodeWithAdditionalFields private long header = 0L; - private BlockInfoContiguous[] blocks; + private BlockInfo[] blocks; INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, - long atime, BlockInfoContiguous[] blklist, short replication, + long atime, BlockInfo[] blklist, short replication, long preferredBlockSize) { this(id, name, permissions, mtime, atime, blklist, replication, preferredBlockSize, (byte) 0); } INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, - long atime, BlockInfoContiguous[] blklist, short replication, + long atime, BlockInfo[] blklist, short replication, long preferredBlockSize, byte storagePolicyID) { super(id, name, permissions, mtime, atime); header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID); @@ -226,13 +226,13 @@ public class INodeFile extends INodeWithAdditionalFields } @Override // BlockCollection - public void setBlock(int index, BlockInfoContiguous blk) { + public void setBlock(int index, BlockInfo blk) { this.blocks[index] = blk; } @Override // BlockCollection, the file should be under construction public BlockInfoContiguousUnderConstruction setLastBlock( - BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations) + BlockInfo lastBlock, DatanodeStorageInfo[] locations) throws IOException { Preconditions.checkState(isUnderConstruction(), "file is no longer under construction"); @@ -265,7 +265,7 @@ public class INodeFile extends INodeWithAdditionalFields BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)blocks[size_1]; //copy to a new list - BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1]; + BlockInfo[] newlist = new BlockInfo[size_1]; System.arraycopy(blocks, 0, newlist, 0, size_1); setBlocks(newlist); return uc; @@ -420,16 +420,16 @@ public class INodeFile extends INodeWithAdditionalFields /** @return the blocks of the file. */ @Override - public BlockInfoContiguous[] getBlocks() { + public BlockInfo[] getBlocks() { return this.blocks; } /** @return blocks of the file corresponding to the snapshot. */ - public BlockInfoContiguous[] getBlocks(int snapshot) { + public BlockInfo[] getBlocks(int snapshot) { if(snapshot == CURRENT_STATE_ID || getDiffs() == null) return getBlocks(); FileDiff diff = getDiffs().getDiffById(snapshot); - BlockInfoContiguous[] snapshotBlocks = + BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks(); if(snapshotBlocks != null) return snapshotBlocks; @@ -441,7 +441,7 @@ public class INodeFile extends INodeWithAdditionalFields void updateBlockCollection() { if (blocks != null) { - for(BlockInfoContiguous b : blocks) { + for(BlockInfo b : blocks) { b.setBlockCollection(this); } } @@ -457,8 +457,8 @@ public class INodeFile extends INodeWithAdditionalFields totalAddedBlocks += f.blocks.length; } - BlockInfoContiguous[] newlist = - new BlockInfoContiguous[size + totalAddedBlocks]; + BlockInfo[] newlist = + new BlockInfo[size + totalAddedBlocks]; System.arraycopy(this.blocks, 0, newlist, 0, size); for(INodeFile in: inodes) { @@ -473,12 +473,12 @@ public class INodeFile extends INodeWithAdditionalFields /** * add a block to the block list */ - void addBlock(BlockInfoContiguous newblock) { + void addBlock(BlockInfo newblock) { if (this.blocks == null) { - this.setBlocks(new BlockInfoContiguous[]{newblock}); + this.setBlocks(new BlockInfo[]{newblock}); } else { int size = this.blocks.length; - BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1]; + BlockInfo[] newlist = new BlockInfo[size + 1]; System.arraycopy(this.blocks, 0, newlist, 0, size); newlist[size] = newblock; this.setBlocks(newlist); @@ -486,7 +486,7 @@ public class INodeFile extends INodeWithAdditionalFields } /** Set the blocks. */ - public void setBlocks(BlockInfoContiguous[] blocks) { + public void setBlocks(BlockInfo[] blocks) { this.blocks = blocks; } @@ -538,7 +538,7 @@ public class INodeFile extends INodeWithAdditionalFields public void clearFile(ReclaimContext reclaimContext) { if (blocks != null && reclaimContext.collectedBlocks != null) { - for (BlockInfoContiguous blk : blocks) { + for (BlockInfo blk : blocks) { reclaimContext.collectedBlocks.addDeleteBlock(blk); blk.setBlockCollection(null); } @@ -710,16 +710,16 @@ public class INodeFile extends INodeWithAdditionalFields */ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build(); - final Iterable blocks; + final Iterable blocks; FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); if (sf == null) { blocks = Arrays.asList(getBlocks()); } else { // Collect all distinct blocks - Set allBlocks = new HashSet<>(Arrays.asList(getBlocks())); + Set allBlocks = new HashSet<>(Arrays.asList(getBlocks())); List diffs = sf.getDiffs().asList(); for(FileDiff diff : diffs) { - BlockInfoContiguous[] diffBlocks = diff.getBlocks(); + BlockInfo[] diffBlocks = diff.getBlocks(); if (diffBlocks != null) { allBlocks.addAll(Arrays.asList(diffBlocks)); } @@ -728,7 +728,7 @@ public class INodeFile extends INodeWithAdditionalFields } final short replication = getPreferredBlockReplication(); - for (BlockInfoContiguous b : blocks) { + for (BlockInfo b : blocks) { long blockSize = b.isComplete() ? b.getNumBytes() : getPreferredBlockSize(); counts.addStorageSpace(blockSize * replication); @@ -747,7 +747,7 @@ public class INodeFile extends INodeWithAdditionalFields /** * Return the penultimate allocated block for this file. */ - BlockInfoContiguous getPenultimateBlock() { + BlockInfo getPenultimateBlock() { if (blocks == null || blocks.length <= 1) { return null; } @@ -755,7 +755,7 @@ public class INodeFile extends INodeWithAdditionalFields } @Override - public BlockInfoContiguous getLastBlock() { + public BlockInfo getLastBlock() { return blocks == null || blocks.length == 0? null: blocks[blocks.length-1]; } @@ -782,7 +782,7 @@ public class INodeFile extends INodeWithAdditionalFields */ public long collectBlocksBeyondMax(final long max, final BlocksMapUpdateInfo collectedBlocks) { - final BlockInfoContiguous[] oldBlocks = getBlocks(); + final BlockInfo[] oldBlocks = getBlocks(); if (oldBlocks == null) return 0; // find the minimum n such that the size of the first n blocks > max @@ -814,17 +814,17 @@ public class INodeFile extends INodeWithAdditionalFields void computeQuotaDeltaForTruncate( long newLength, BlockStoragePolicy bsps, QuotaCounts delta) { - final BlockInfoContiguous[] blocks = getBlocks(); + final BlockInfo[] blocks = getBlocks(); if (blocks == null || blocks.length == 0) { return; } long size = 0; - for (BlockInfoContiguous b : blocks) { + for (BlockInfo b : blocks) { size += b.getNumBytes(); } - BlockInfoContiguous[] sblocks = null; + BlockInfo[] sblocks = null; FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); if (sf != null) { FileDiff diff = sf.getDiffs().getLast(); @@ -833,7 +833,7 @@ public class INodeFile extends INodeWithAdditionalFields for (int i = blocks.length - 1; i >= 0 && size > newLength; size -= blocks[i].getNumBytes(), --i) { - BlockInfoContiguous bi = blocks[i]; + BlockInfo bi = blocks[i]; long truncatedBytes; if (size - newLength < bi.getNumBytes()) { // Record a full block as the last block will be copied during @@ -863,20 +863,20 @@ public class INodeFile extends INodeWithAdditionalFields } void truncateBlocksTo(int n) { - final BlockInfoContiguous[] newBlocks; + final BlockInfo[] newBlocks; if (n == 0) { - newBlocks = BlockInfoContiguous.EMPTY_ARRAY; + newBlocks = BlockInfo.EMPTY_ARRAY; } else { - newBlocks = new BlockInfoContiguous[n]; + newBlocks = new BlockInfo[n]; System.arraycopy(getBlocks(), 0, newBlocks, 0, n); } // set new blocks setBlocks(newBlocks); } - public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks, + public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks, BlocksMapUpdateInfo collectedBlocks) { - BlockInfoContiguous[] oldBlocks = getBlocks(); + BlockInfo[] oldBlocks = getBlocks(); if(snapshotBlocks == null || oldBlocks == null) return; // Skip blocks in common between the file and the snapshot @@ -900,7 +900,7 @@ public class INodeFile extends INodeWithAdditionalFields FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); if(sf == null) return; - BlockInfoContiguous[] snapshotBlocks = + BlockInfo[] snapshotBlocks = getDiffs().findEarlierSnapshotBlocks(snapshotId); if(snapshotBlocks == null) return; @@ -914,12 +914,12 @@ public class INodeFile extends INodeWithAdditionalFields /** * @return true if the block is contained in a snapshot or false otherwise. */ - boolean isBlockInLatestSnapshot(BlockInfoContiguous block) { + boolean isBlockInLatestSnapshot(BlockInfo block) { FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature(); if (sf == null || sf.getDiffs() == null) { return false; } - BlockInfoContiguous[] snapshotBlocks = getDiffs() + BlockInfo[] snapshotBlocks = getDiffs() .findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId()); return snapshotBlocks != null && Arrays.asList(snapshotBlocks).contains(block); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java index 0806f82..f954a58 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java @@ -33,7 +33,7 @@ import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.util.Daemon; @@ -108,11 +108,11 @@ public class LeaseManager { for (Long id : getINodeIdWithLeases()) { final INodeFile cons = fsnamesystem.getFSDirectory().getInode(id).asFile(); Preconditions.checkState(cons.isUnderConstruction()); - BlockInfoContiguous[] blocks = cons.getBlocks(); + BlockInfo[] blocks = cons.getBlocks(); if(blocks == null) { continue; } - for(BlockInfoContiguous b : blocks) { + for(BlockInfo b : blocks) { if(!b.isComplete()) numUCBlocks++; } http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java index 0daf367..39180b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java @@ -64,7 +64,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus; @@ -243,7 +243,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { //get blockInfo Block block = new Block(Block.getBlockId(blockId)); //find which file this block belongs to - BlockInfoContiguous blockInfo = bm.getStoredBlock(block); + BlockInfo blockInfo = bm.getStoredBlock(block); if(blockInfo == null) { out.println("Block "+ blockId +" " + NONEXISTENT_STATUS); LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java index c4cbbc1..363c208 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat; import org.apache.hadoop.hdfs.server.namenode.AclFeature; import org.apache.hadoop.hdfs.server.namenode.FSDirectory; @@ -239,13 +239,13 @@ public class FSImageFormatPBSnapshot { FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, pbf.getFileSize()); List bpl = pbf.getBlocksList(); - BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()]; + BlockInfo[] blocks = new BlockInfo[bpl.size()]; for(int j = 0, e = bpl.size(); j < e; ++j) { Block blk = PBHelper.convert(bpl.get(j)); - BlockInfoContiguous storedBlock = fsn.getBlockManager().getStoredBlock(blk); + BlockInfo storedBlock = fsn.getBlockManager().getStoredBlock(blk); if(storedBlock == null) { storedBlock = fsn.getBlockManager().addBlockCollection( - new BlockInfoContiguous(blk, copy.getFileReplication()), file); + new BlockInfo(blk, copy.getFileReplication()), file); } blocks[j] = storedBlock; } http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java index 48c5f33..52bf58e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java @@ -21,7 +21,7 @@ import java.io.DataOutput; import java.io.IOException; import java.util.Arrays; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; @@ -38,7 +38,7 @@ public class FileDiff extends /** The file size at snapshot creation time. */ private final long fileSize; /** A copy of the INodeFile block list. Used in truncate. */ - private BlockInfoContiguous[] blocks; + private BlockInfo[] blocks; FileDiff(int snapshotId, INodeFile file) { super(snapshotId, null, null); @@ -64,7 +64,7 @@ public class FileDiff extends * up to the current {@link #fileSize}. * Should be done only once. */ - public void setBlocks(BlockInfoContiguous[] blocks) { + public void setBlocks(BlockInfo[] blocks) { if(this.blocks != null) return; int numBlocks = 0; @@ -73,7 +73,7 @@ public class FileDiff extends this.blocks = Arrays.copyOf(blocks, numBlocks); } - public BlockInfoContiguous[] getBlocks() { + public BlockInfo[] getBlocks() { return blocks; } @@ -118,7 +118,7 @@ public class FileDiff extends if (blocks == null || collectedBlocks == null) { return; } - for (BlockInfoContiguous blk : blocks) { + for (BlockInfo blk : blocks) { collectedBlocks.addDeleteBlock(blk); } blocks = null; http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java index 62aaccd..6b8388e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java @@ -21,7 +21,7 @@ import java.util.Collections; import java.util.List; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.INode; @@ -59,14 +59,14 @@ public class FileDiffList extends } } - public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) { + public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) { assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id"; if (snapshotId == Snapshot.CURRENT_STATE_ID) { return null; } List diffs = this.asList(); int i = Collections.binarySearch(diffs, snapshotId); - BlockInfoContiguous[] blocks = null; + BlockInfo[] blocks = null; for(i = i >= 0 ? i : -i-2; i >= 0; i--) { blocks = diffs.get(i).getBlocks(); if(blocks != null) { @@ -76,14 +76,14 @@ public class FileDiffList extends return blocks; } - public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) { + public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) { assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id"; if (snapshotId == Snapshot.CURRENT_STATE_ID) { return null; } List diffs = this.asList(); int i = Collections.binarySearch(diffs, snapshotId); - BlockInfoContiguous[] blocks = null; + BlockInfo[] blocks = null; for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) { blocks = diffs.get(i).getBlocks(); if (blocks != null) { @@ -100,7 +100,7 @@ public class FileDiffList extends */ void combineAndCollectSnapshotBlocks( INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) { - BlockInfoContiguous[] removedBlocks = removed.getBlocks(); + BlockInfo[] removedBlocks = removed.getBlocks(); if (removedBlocks == null) { FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature(); assert sf != null : "FileWithSnapshotFeature is null"; @@ -114,10 +114,10 @@ public class FileDiffList extends if (earlierDiff != null) { earlierDiff.setBlocks(removedBlocks); } - BlockInfoContiguous[] earlierBlocks = - (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks()); + BlockInfo[] earlierBlocks = + (earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks()); // Find later snapshot (or file itself) with blocks - BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId()); + BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId()); laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks; // Skip blocks, which belong to either the earlier or the later lists int i = 0; @@ -129,7 +129,7 @@ public class FileDiffList extends break; } // Check if last block is part of truncate recovery - BlockInfoContiguous lastBlock = file.getLastBlock(); + BlockInfo lastBlock = file.getLastBlock(); Block dontRemoveBlock = null; if (lastBlock != null && lastBlock.getBlockUCState().equals( HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) { http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java index 555a662..f017d4b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java @@ -22,7 +22,7 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.namenode.AclFeature; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.AclStorage; @@ -208,7 +208,7 @@ public class FileWithSnapshotFeature implements INode.Feature { // Collect blocks that should be deleted FileDiff last = diffs.getLast(); - BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks(); + BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks(); if(snapshotBlocks == null) file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks()); else http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java ---------------------------------------------------------------------- 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 a88a459..50b85c0 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 @@ -108,7 +108,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; @@ -1609,7 +1609,7 @@ public class DFSTestUtil { public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn, ExtendedBlock blk) { BlockManager bm0 = nn.getNamesystem().getBlockManager(); - BlockInfoContiguous storedBlock = bm0.getStoredBlock(blk.getLocalBlock()); + BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock()); assertTrue("Block " + blk + " should be under construction, " + "got: " + storedBlock, storedBlock instanceof BlockInfoContiguousUnderConstruction); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java index 1ab7427..1f18014 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -671,7 +671,7 @@ public class TestDecommission { GenericTestUtils.waitFor(new Supplier() { @Override public Boolean get() { - BlockInfoContiguous info = + BlockInfo info = blockManager.getStoredBlock(b.getLocalBlock()); int count = 0; StringBuilder sb = new StringBuilder("Replica locations: "); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java ---------------------------------------------------------------------- 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 c5662ca..96d4d8b 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 @@ -49,7 +49,7 @@ public class TestBlockInfo { @Test public void testIsDeleted() { - BlockInfoContiguous blockInfo = new BlockInfoContiguous((short) 3); + BlockInfo blockInfo = new BlockInfo((short) 3); BlockCollection bc = Mockito.mock(BlockCollection.class); blockInfo.setBlockCollection(bc); Assert.assertFalse(blockInfo.isDeleted()); @@ -59,7 +59,7 @@ public class TestBlockInfo { @Test public void testAddStorage() throws Exception { - BlockInfoContiguous blockInfo = new BlockInfoContiguous((short) 3); + BlockInfo blockInfo = new BlockInfo((short) 3); final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1"); @@ -71,9 +71,9 @@ public class TestBlockInfo { @Test public void testCopyConstructor() { - BlockInfoContiguous old = new BlockInfoContiguous((short) 3); + BlockInfo old = new BlockInfo((short) 3); try { - BlockInfoContiguous copy = new BlockInfoContiguous(old); + BlockInfo copy = new BlockInfo(old); assertEquals(old.getBlockCollection(), copy.getBlockCollection()); assertEquals(old.getCapacity(), copy.getCapacity()); } catch (Exception e) { @@ -88,11 +88,11 @@ public class TestBlockInfo { final DatanodeStorageInfo storage1 = DFSTestUtil.createDatanodeStorageInfo("storageID1", "127.0.0.1"); final DatanodeStorageInfo storage2 = new DatanodeStorageInfo(storage1.getDatanodeDescriptor(), new DatanodeStorage("storageID2")); final int NUM_BLOCKS = 10; - BlockInfoContiguous[] blockInfos = new BlockInfoContiguous[NUM_BLOCKS]; + BlockInfo[] blockInfos = new BlockInfo[NUM_BLOCKS]; // Create a few dummy blocks and add them to the first storage. for (int i = 0; i < NUM_BLOCKS; ++i) { - blockInfos[i] = new BlockInfoContiguous((short) 3); + blockInfos[i] = new BlockInfo((short) 3); storage1.addBlock(blockInfos[i]); } @@ -111,14 +111,14 @@ public class TestBlockInfo { DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1"); ArrayList blockList = new ArrayList(MAX_BLOCKS); - ArrayList blockInfoList = new ArrayList(); + ArrayList blockInfoList = new ArrayList(); int headIndex; int curIndex; LOG.info("Building block list..."); for (int i = 0; i < MAX_BLOCKS; i++) { blockList.add(new Block(i, 0, GenerationStamp.LAST_RESERVED_STAMP)); - blockInfoList.add(new BlockInfoContiguous(blockList.get(i), (short) 3)); + blockInfoList.add(new BlockInfo(blockList.get(i), (short) 3)); dd.addBlock(blockInfoList.get(i)); // index of the datanode should be 0 @@ -129,7 +129,7 @@ public class TestBlockInfo { // list length should be equal to the number of blocks we inserted LOG.info("Checking list length..."); assertEquals("Length should be MAX_BLOCK", MAX_BLOCKS, dd.numBlocks()); - Iterator it = dd.getBlockIterator(); + Iterator it = dd.getBlockIterator(); int len = 0; while (it.hasNext()) { it.next(); @@ -151,7 +151,7 @@ public class TestBlockInfo { // move head of the list to the head - this should not change the list LOG.info("Moving head to the head..."); - BlockInfoContiguous temp = dd.getBlockListHeadForTesting(); + BlockInfo temp = dd.getBlockListHeadForTesting(); curIndex = 0; headIndex = 0; dd.moveBlockToHead(temp, curIndex, headIndex); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java ---------------------------------------------------------------------- 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 58210c1..bf46ed7 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 @@ -147,7 +147,7 @@ public class TestBlockManager { private void doBasicTest(int testIndex) { List origStorages = getStorages(0, 1); List origNodes = getNodes(origStorages); - BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo); assertEquals(2, pipeline.length); @@ -179,7 +179,7 @@ public class TestBlockManager { // Block originally on A1, A2, B1 List origStorages = getStorages(0, 1, 3); List origNodes = getNodes(origStorages); - BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); // Decommission two of the nodes (A1, A2) List decomNodes = startDecommission(0, 1); @@ -223,7 +223,7 @@ public class TestBlockManager { // Block originally on A1, A2, B1 List origStorages = getStorages(0, 1, 3); List origNodes = getNodes(origStorages); - BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); // Decommission all of the nodes List decomNodes = startDecommission(0, 1, 3); @@ -276,7 +276,7 @@ public class TestBlockManager { // Block originally on A1, A2, B1 List origStorages = getStorages(0, 1, 3); List origNodes = getNodes(origStorages); - BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); // Decommission all of the nodes in rack A List decomNodes = startDecommission(0, 1, 2); @@ -335,7 +335,7 @@ public class TestBlockManager { private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) { // Originally on only nodes in rack A. List origNodes = rackA; - BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes); + BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes); DatanodeStorageInfo pipeline[] = scheduleSingleReplication(blockInfo); assertEquals(2, pipeline.length); // single new copy @@ -378,7 +378,7 @@ public class TestBlockManager { * Tell the block manager that replication is completed for the given * pipeline. */ - private void fulfillPipeline(BlockInfoContiguous blockInfo, + private void fulfillPipeline(BlockInfo blockInfo, DatanodeStorageInfo[] pipeline) throws IOException { for (int i = 1; i < pipeline.length; i++) { DatanodeStorageInfo storage = pipeline[i]; @@ -387,9 +387,9 @@ public class TestBlockManager { } } - private BlockInfoContiguous blockOnNodes(long blkId, List nodes) { + private BlockInfo blockOnNodes(long blkId, List nodes) { Block block = new Block(blkId); - BlockInfoContiguous blockInfo = new BlockInfoContiguous(block, (short) 3); + BlockInfo blockInfo = new BlockInfo(block, (short) 3); for (DatanodeDescriptor dn : nodes) { for (DatanodeStorageInfo storage : dn.getStorageInfos()) { @@ -431,10 +431,10 @@ public class TestBlockManager { return nodes; } - private BlockInfoContiguous addBlockOnNodes(long blockId, List nodes) { + private BlockInfo addBlockOnNodes(long blockId, List nodes) { BlockCollection bc = Mockito.mock(BlockCollection.class); Mockito.doReturn((short)3).when(bc).getPreferredBlockReplication(); - BlockInfoContiguous blockInfo = blockOnNodes(blockId, nodes); + BlockInfo blockInfo = blockOnNodes(blockId, nodes); bm.blocksMap.addBlockCollection(blockInfo, bc); return blockInfo; @@ -677,21 +677,21 @@ public class TestBlockManager { // blk_42 is finalized. long receivedBlockId = 42; // arbitrary - BlockInfoContiguous receivedBlock = addBlockToBM(receivedBlockId); + BlockInfo receivedBlock = addBlockToBM(receivedBlockId); rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivedBlock), ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null)); builder.add(new FinalizedReplica(receivedBlock, null, null)); // blk_43 is under construction. long receivingBlockId = 43; - BlockInfoContiguous receivingBlock = addUcBlockToBM(receivingBlockId); + BlockInfo receivingBlock = addUcBlockToBM(receivingBlockId); rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivingBlock), ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, null)); builder.add(new ReplicaBeingWritten(receivingBlock, null, null, null)); // blk_44 has 2 records in IBR. It's finalized. So full BR has 1 record. long receivingReceivedBlockId = 44; - BlockInfoContiguous receivingReceivedBlock = addBlockToBM(receivingReceivedBlockId); + BlockInfo receivingReceivedBlock = addBlockToBM(receivingReceivedBlockId); rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivingReceivedBlock), ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, null)); rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivingReceivedBlock), @@ -709,7 +709,7 @@ public class TestBlockManager { // blk_46 exists in DN for a long time, so it's in full BR, but not in IBR. long existedBlockId = 46; - BlockInfoContiguous existedBlock = addBlockToBM(existedBlockId); + BlockInfo existedBlock = addBlockToBM(existedBlockId); builder.add(new FinalizedReplica(existedBlock, null, null)); // process IBR and full BR @@ -735,17 +735,17 @@ public class TestBlockManager { (ds) >= 0); } - private BlockInfoContiguous addBlockToBM(long blkId) { + private BlockInfo addBlockToBM(long blkId) { Block block = new Block(blkId); - BlockInfoContiguous blockInfo = - new BlockInfoContiguous(block, (short) 3); + BlockInfo blockInfo = + new BlockInfo(block, (short) 3); BlockCollection bc = Mockito.mock(BlockCollection.class); Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication(); bm.blocksMap.addBlockCollection(blockInfo, bc); return blockInfo; } - private BlockInfoContiguous addUcBlockToBM(long blkId) { + private BlockInfo addUcBlockToBM(long blkId) { Block block = new Block(blkId); BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(block, (short) 3); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java ---------------------------------------------------------------------- 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 7cdb423..fe639e4 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 @@ -58,8 +58,8 @@ public class TestDatanodeDescriptor { public void testBlocksCounter() throws Exception { DatanodeDescriptor dd = BlockManagerTestUtil.getLocalDatanodeDescriptor(true); assertEquals(0, dd.numBlocks()); - BlockInfoContiguous blk = new BlockInfoContiguous(new Block(1L), (short) 1); - BlockInfoContiguous blk1 = new BlockInfoContiguous(new Block(2L), (short) 2); + BlockInfo blk = new BlockInfo(new Block(1L), (short) 1); + BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2); DatanodeStorageInfo[] storages = dd.getStorageInfos(); assertTrue(storages.length > 0); // add first block http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java ---------------------------------------------------------------------- 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 259404e..844c5d0 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 @@ -159,7 +159,7 @@ public class TestPendingReplication { DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, TIMEOUT); MiniDFSCluster cluster = null; Block block; - BlockInfoContiguous blockInfo; + BlockInfo blockInfo; try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_COUNT).build(); @@ -178,7 +178,7 @@ public class TestPendingReplication { // block = new Block(1, 1, 0); - blockInfo = new BlockInfoContiguous(block, (short) 3); + blockInfo = new BlockInfo(block, (short) 3); pendingReplications.increment(block, DatanodeStorageInfo.toDatanodeDescriptors( http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java ---------------------------------------------------------------------- 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 3226578..f317723 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 @@ -34,7 +34,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; @@ -1219,7 +1218,7 @@ public class TestReplicationPolicy { chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1); assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0); - final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1); + final BlockInfo info = new BlockInfo(block1, (short) 1); final BlockCollection mbc = mock(BlockCollection.class); when(mbc.getLastBlock()).thenReturn(info); when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1); @@ -1241,12 +1240,12 @@ public class TestReplicationPolicy { when(dn.isDecommissioned()).thenReturn(true); when(storage.getState()).thenReturn(DatanodeStorage.State.NORMAL); when(storage.getDatanodeDescriptor()).thenReturn(dn); - when(storage.removeBlock(any(BlockInfoContiguous.class))).thenReturn(true); - when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn + when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true); + when(storage.addBlock(any(BlockInfo.class))).thenReturn (DatanodeStorageInfo.AddBlockResult.ADDED); ucBlock.addStorage(storage); - when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any())) + when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any())) .thenReturn(ucBlock); bm.convertLastBlockToUnderConstruction(mbc, 0L); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java index 0349251..3f96c0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java @@ -23,7 +23,7 @@ import java.io.IOException; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.Storage; @@ -66,10 +66,10 @@ public class CreateEditsLog { INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p, 0L); editLog.logMkDir(BASE_PATH, dirInode); - BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile]; + BlockInfo[] blocks = new BlockInfo[blocksPerFile]; for (int iB = 0; iB < blocksPerFile; ++iB) { blocks[iB] = - new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP), + new BlockInfo(new Block(0, blockSize, BLOCK_GENERATION_STAMP), replication); } @@ -97,7 +97,7 @@ public class CreateEditsLog { editLog.logMkDir(currentDir, dirInode); } INodeFile fileUc = new INodeFile(inodeId.nextValue(), null, - p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize); + p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize); fileUc.toUnderConstruction("", ""); editLog.logOpenFile(filePath, fileUc, false, false); editLog.logCloseFile(filePath, inode); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java index a417c3d..301ee25 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.junit.After; import org.junit.Before; @@ -87,21 +87,21 @@ public class TestAddBlock { // check file1 INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile(); - BlockInfoContiguous[] file1Blocks = file1Node.getBlocks(); + BlockInfo[] file1Blocks = file1Node.getBlocks(); assertEquals(1, file1Blocks.length); assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes()); assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState()); // check file2 INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile(); - BlockInfoContiguous[] file2Blocks = file2Node.getBlocks(); + BlockInfo[] file2Blocks = file2Node.getBlocks(); assertEquals(1, file2Blocks.length); assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes()); assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState()); // check file3 INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile(); - BlockInfoContiguous[] file3Blocks = file3Node.getBlocks(); + BlockInfo[] file3Blocks = file3Node.getBlocks(); assertEquals(2, file3Blocks.length); assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes()); assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState()); @@ -110,7 +110,7 @@ public class TestAddBlock { // check file4 INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile(); - BlockInfoContiguous[] file4Blocks = file4Node.getBlocks(); + BlockInfo[] file4Blocks = file4Node.getBlocks(); assertEquals(2, file4Blocks.length); assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes()); assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState()); @@ -141,7 +141,7 @@ public class TestAddBlock { FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile(); - BlockInfoContiguous[] fileBlocks = fileNode.getBlocks(); + BlockInfo[] fileBlocks = fileNode.getBlocks(); assertEquals(2, fileBlocks.length); assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes()); assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState()); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java index 1fbe160..f372bec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.TestFileCreation; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; @@ -91,12 +91,12 @@ public class TestBlockUnderConstruction { " isUnderConstruction = " + inode.isUnderConstruction() + " expected to be " + isFileOpen, inode.isUnderConstruction() == isFileOpen); - BlockInfoContiguous[] blocks = inode.getBlocks(); + BlockInfo[] blocks = inode.getBlocks(); assertTrue("File does not have blocks: " + inode.toString(), blocks != null && blocks.length > 0); int idx = 0; - BlockInfoContiguous curBlock; + BlockInfo curBlock; // all blocks but the last two should be regular blocks for(; idx < blocks.length - 2; idx++) { curBlock = blocks[idx]; http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java ---------------------------------------------------------------------- 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 ea560fe..8c9da01 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 @@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -78,7 +78,7 @@ public class TestCommitBlockSynchronization { doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class)); doReturn(blockInfo).when(file).getLastBlock(); doReturn("").when(namesystemSpy).closeFileCommitBlocks( - any(INodeFile.class), any(BlockInfoContiguous.class)); + any(INodeFile.class), any(BlockInfo.class)); doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog(); return namesystemSpy; @@ -106,7 +106,7 @@ public class TestCommitBlockSynchronization { lastBlock, genStamp, length, false, false, newTargets, null); // Simulate 'completing' the block. - BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1); + BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1); completedBlockInfo.setBlockCollection(file); completedBlockInfo.setGenerationStamp(genStamp); doReturn(completedBlockInfo).when(namesystemSpy) @@ -178,7 +178,7 @@ public class TestCommitBlockSynchronization { namesystemSpy.commitBlockSynchronization( lastBlock, genStamp, length, true, false, newTargets, null); - BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1); + BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1); completedBlockInfo.setBlockCollection(file); completedBlockInfo.setGenerationStamp(genStamp); doReturn(completedBlockInfo).when(namesystemSpy) http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java index 1e42e34..f654107 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java @@ -69,7 +69,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; @@ -205,7 +205,7 @@ public class TestEditLog { for (int i = 0; i < numTransactions; i++) { INodeFile inode = new INodeFile(namesystem.dir.allocateNewInodeId(), null, - p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize); + p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize); inode.toUnderConstruction("", ""); editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false); http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java index 5653df5..27a1bd3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease; @@ -105,7 +105,7 @@ public class TestFSImage { INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile(); assertEquals("hello".length(), file2Node.computeFileSize()); assertTrue(file2Node.isUnderConstruction()); - BlockInfoContiguous[] blks = file2Node.getBlocks(); + BlockInfo[] blks = file2Node.getBlocks(); assertEquals(1, blks.length); assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState()); // check lease manager http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index 7f31f84..eabd0c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java @@ -79,7 +79,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 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.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; @@ -885,7 +885,7 @@ public class TestFsck { // intentionally corrupt NN data structure INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode (fileName, true); - final BlockInfoContiguous[] blocks = node.getBlocks(); + final BlockInfo[] blocks = node.getBlocks(); assertEquals(blocks.length, 1); blocks[0].setNumBytes(-1L); // set the block length to be negative http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java index 4d0f994..7a3a8d6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java @@ -21,7 +21,7 @@ import org.apache.commons.io.Charsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -125,7 +125,7 @@ public class TestGetBlockLocations { FsPermission.createImmutable((short) 0x1ff)); final INodeFile file = new INodeFile( MOCK_INODE_ID, FILE_NAME.getBytes(Charsets.UTF_8), - perm, 1, 1, new BlockInfoContiguous[] {}, (short) 1, + perm, 1, 1, new BlockInfo[] {}, (short) 1, DFS_BLOCK_SIZE_DEFAULT); fsn.getFSDirectory().addINode(iip, file); return fsn; http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b88df72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java index b45d2f6..21b67a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java @@ -60,7 +60,7 @@ 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.QuotaExceededException; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.io.IOUtils; @@ -290,7 +290,7 @@ public class TestINodeFile { iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication, preferredBlockSize); iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i)); - BlockInfoContiguous newblock = new BlockInfoContiguous(replication); + BlockInfo newblock = new BlockInfo(replication); iNodes[i].addBlock(newblock); }