From f166e67a23bc49e932b23876ede6f8ab9c9f76d6 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Thu, 6 Nov 2014 10:03:26 +0530 Subject: [PATCH 001/212] HDFS-7347. Configurable erasure coding policy for individual files and directories ( Contributed by Zhe Zhang ) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 4 + .../hadoop/hdfs/TestBlockStoragePolicy.java | 3 + .../TestBlockInitialEncoding.java | 75 +++++++++++++++++++ 3 files changed, 82 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt new file mode 100644 index 0000000000000..2ef8527334aae --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -0,0 +1,4 @@ + BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS + + HDFS-7347. Configurable erasure coding policy for individual files and + directories ( Zhe Zhang via vinayakumarb ) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java index ea69f976a6f16..13214f34484ae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java @@ -119,6 +119,9 @@ public void testDefaultPolicies() { expectedPolicyStrings.put(COLD, "BlockStoragePolicy{COLD:" + COLD + ", storageTypes=[ARCHIVE], " + "creationFallbacks=[], replicationFallbacks=[]}"); + expectedPolicyStrings.put(EC, + "BlockStoragePolicy{EC:" + EC + ", storageTypes=[DISK], " + + "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}"); expectedPolicyStrings.put(WARM, "BlockStoragePolicy{WARM:" + WARM + ", storageTypes=[DISK, ARCHIVE], " + "creationFallbacks=[DISK, ARCHIVE], " + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java new file mode 100644 index 0000000000000..a84f67b232b0c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java @@ -0,0 +1,75 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.*; +import org.apache.hadoop.hdfs.client.HdfsAdmin; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INode; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_ID; +import static org.junit.Assert.assertEquals; + +public class TestBlockInitialEncoding { + private final int NUM_OF_DATANODES = 3; + private Configuration conf; + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private static final int BLOCK_SIZE = 1024; + private HdfsAdmin dfsAdmin; + private FSNamesystem namesystem; + + @Before + public void setupCluster() throws IOException { + conf = new HdfsConfiguration(); + conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + cluster = new MiniDFSCluster.Builder(conf). + numDataNodes(NUM_OF_DATANODES).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + dfsAdmin = new HdfsAdmin(cluster.getURI(), conf); + namesystem = cluster.getNamesystem(); + } + + @After + public void shutdownCluster() throws IOException { + cluster.shutdown(); + } + + @Test + public void testBlockInitialEncoding() + throws IOException, InterruptedException { + final Path testDir = new Path("/test"); + fs.mkdir(testDir, FsPermission.getDirDefault()); + dfsAdmin.setStoragePolicy(testDir, EC_STORAGE_POLICY_NAME); + final Path ECFilePath = new Path("/test/foo.ec"); + DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, (short) 3, 0); + INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString()); + assertEquals(EC_STORAGE_POLICY_ID, inode.getStoragePolicyID()); + } + +} \ No newline at end of file From bc2833b1c91e107d090619d755c584f6eae82327 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 30 Jan 2015 16:16:26 -0800 Subject: [PATCH 002/212] HDFS-7339. Allocating and persisting block groups in NameNode. Contributed by Zhe Zhang --- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 + .../blockmanagement/BlockIdManager.java | 8 +- .../SequentialBlockGroupIdGenerator.java | 82 ++++++++++++++++++ .../SequentialBlockIdGenerator.java | 6 +- .../hdfs/server/namenode/FSNamesystem.java | 11 ++- .../hdfs/server/namenode/INodeFile.java | 11 +++ .../server/namenode/TestAddBlockgroup.java | 84 +++++++++++++++++++ 7 files changed, 198 insertions(+), 6 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 9c19f91908aff..bd86964a17c46 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -159,6 +159,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT = 3; public static final String DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min"; public static final int DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1; + public static final String DFS_NAMENODE_STRIPE_MIN_KEY = "dfs.namenode.stripe.min"; + public static final int DFS_NAMENODE_STRIPE_MIN_DEFAULT = 1; public static final String DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY = "dfs.namenode.replication.pending.timeout-sec"; public static final int DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT = -1; public static final String DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY = "dfs.namenode.replication.max-streams"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index feebd87fa73ce..a2f8fcc57136c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -54,10 +54,12 @@ public class BlockIdManager { * The global block ID space for this file system. */ private final SequentialBlockIdGenerator blockIdGenerator; + private final SequentialBlockGroupIdGenerator blockGroupIdGenerator; public BlockIdManager(BlockManager blockManager) { this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP; this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager); + this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager); } /** @@ -191,6 +193,10 @@ public long nextBlockId() { return blockIdGenerator.nextValue(); } + public long nextBlockGroupId() { + return blockGroupIdGenerator.nextValue(); + } + public boolean isGenStampInFuture(Block block) { if (isLegacyBlock(block)) { return block.getGenerationStamp() > getGenerationStampV1(); @@ -206,4 +212,4 @@ public void clear() { .LAST_RESERVED_BLOCK_ID); generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP; } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java new file mode 100644 index 0000000000000..e9e22ee46d539 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java @@ -0,0 +1,82 @@ +/** + * 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 org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.util.SequentialNumber; + +/** + * Generate the next valid block group ID by incrementing the maximum block + * group ID allocated so far, with the first 2^10 block group IDs reserved. + * HDFS-EC introduces a hierarchical protocol to name blocks and groups: + * Contiguous: {reserved block IDs | flag | block ID} + * Striped: {reserved block IDs | flag | block group ID | index in group} + * + * Following n bits of reserved block IDs, The (n+1)th bit in an ID + * distinguishes contiguous (0) and striped (1) blocks. For a striped block, + * bits (n+2) to (64-m) represent the ID of its block group, while the last m + * bits represent its index of the group. The value m is determined by the + * maximum number of blocks in a group (MAX_BLOCKS_IN_GROUP). + */ +@InterfaceAudience.Private +public class SequentialBlockGroupIdGenerator extends SequentialNumber { + + private final BlockManager blockManager; + + SequentialBlockGroupIdGenerator(BlockManager blockManagerRef) { + super(Long.MIN_VALUE); + this.blockManager = blockManagerRef; + } + + @Override // NumberGenerator + public long nextValue() { + // Skip to next legitimate block group ID based on the naming protocol + while (super.getCurrentValue() % HdfsConstants.MAX_BLOCKS_IN_GROUP > 0) { + super.nextValue(); + } + // Make sure there's no conflict with existing random block IDs + while (hasValidBlockInRange(super.getCurrentValue())) { + super.skipTo(super.getCurrentValue() + + HdfsConstants.MAX_BLOCKS_IN_GROUP); + } + if (super.getCurrentValue() >= 0) { + BlockManager.LOG.warn("All negative block group IDs are used, " + + "growing into positive IDs, " + + "which might conflict with non-erasure coded blocks."); + } + return super.getCurrentValue(); + } + + /** + * + * @param id The starting ID of the range + * @return true if any ID in the range + * {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a file + */ + private boolean hasValidBlockInRange(long id) { + for (int i = 0; i < HdfsConstants.MAX_BLOCKS_IN_GROUP; i++) { + Block b = new Block(id + i); + if (blockManager.getBlockCollection(b) != null) { + return true; + } + } + return false; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java index eef8857b042f4..c97de4b806aaf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java @@ -19,7 +19,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.util.SequentialNumber; /** @@ -54,6 +53,11 @@ public long nextValue() { while(isValidBlock(b)) { b.setBlockId(super.nextValue()); } + if (b.getBlockId() < 0) { + BlockManager.LOG.warn("All positive block IDs are used, " + + "wrapping to negative IDs, " + + "which might conflict with erasure coded block groups."); + } return b.getBlockId(); } 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 bfd6ebaac12e3..0d229e1af451d 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 @@ -2093,7 +2093,7 @@ Block prepareFileForTruncate(INodesInPath iip, BlockInfoContiguous oldBlock = file.getLastBlock(); boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock); if(newBlock == null) { - newBlock = (shouldCopyOnTruncate) ? createNewBlock() : + newBlock = (shouldCopyOnTruncate) ? createNewBlock(file.isStriped()) : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(), nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock))); } @@ -3044,10 +3044,11 @@ boolean completeFile(final String src, String holder, /** * Create new block with a unique block id and a new generation stamp. + * @param isStriped is the file under striping or contiguous layout? */ Block createNewBlock() throws IOException { assert hasWriteLock(); - Block b = new Block(nextBlockId(), 0, 0); + Block b = new Block(nextBlockId(isStriped), 0, 0); // Increment the generation stamp for every new block. b.setGenerationStamp(nextGenerationStamp(false)); return b; @@ -5610,11 +5611,13 @@ long nextGenerationStamp(boolean legacyBlock) /** * Increments, logs and then returns the block ID + * @param isStriped is the file under striping or contiguous layout? */ - private long nextBlockId() throws IOException { + private long nextBlockId(boolean isStriped) throws IOException { assert hasWriteLock(); checkNameNodeSafeMode("Cannot get next block ID"); - final long blockId = blockIdManager.nextBlockId(); + final long blockId = isStriped ? + blockIdManager.nextBlockGroupId() : blockIdManager.nextBlockId(); getEditLog().logAllocateBlockId(blockId); // NB: callers sync the log return blockId; 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 a6ff6fb89f432..dec8a3b798d26 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 @@ -34,12 +34,14 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; 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.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList; @@ -924,4 +926,13 @@ boolean isBlockInLatestSnapshot(BlockInfoContiguous block) { return snapshotBlocks != null && Arrays.asList(snapshotBlocks).contains(block); } + + @VisibleForTesting + /** + * @return true if the file is in the striping layout. + */ + // TODO: move erasure coding policy to file XAttr (HDFS-7337) + public boolean isStriped() { + return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java new file mode 100644 index 0000000000000..95133ce77c234 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +public class TestAddBlockgroup { + + public static final Log LOG = LogFactory.getLog(TestAddBlockgroup.class); + + private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; + private final short NUM_DATANODES = GROUP_SIZE; + + private static final int BLOCKSIZE = 1024; + private static final short REPLICATION = 3; + + private MiniDFSCluster cluster; + private Configuration conf; + + @Before + public void setup() throws IOException { + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES) + .build(); + cluster.waitActive(); + cluster.getFileSystem().setStoragePolicy(new Path("/"), + HdfsConstants.EC_STORAGE_POLICY_NAME); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testAddBlockGroup() throws Exception { + DistributedFileSystem fs = cluster.getFileSystem(); + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + + final Path file1 = new Path("/file1"); + DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L); + INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile(); + BlockInfo[] file1Blocks = file1Node.getBlocks(); + assertEquals(2, file1Blocks.length); + assertEquals(GROUP_SIZE, file1Blocks[0].numNodes()); + assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP, + file1Blocks[1].getBlockId() - file1Blocks[0].getBlockId()); + } +} From 0c1da5a0300f015a7e39f2b40a73fb06c65a78c8 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 9 Feb 2015 10:27:14 -0800 Subject: [PATCH 003/212] HDFS-7652. Process block reports for erasure coded blocks. Contributed by Zhe Zhang --- .../server/blockmanagement/BlockIdManager.java | 8 ++++++++ .../server/blockmanagement/BlockManager.java | 18 +++++++++++++----- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index a2f8fcc57136c..230d9a5d1e937 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -212,4 +212,12 @@ public void clear() { .LAST_RESERVED_BLOCK_ID); generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP; } + + public static boolean isStripedBlockID(long id) { + return id < 0; + } + + public static long convertToGroupID(long id) { + return id & (~(HdfsConstants.MAX_BLOCKS_IN_GROUP - 1)); + } } 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 54981fb4a687d..dda19434b84ec 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 @@ -1943,7 +1943,7 @@ void rescanPostponedMisreplicatedBlocks() { break; } - BlockInfoContiguous bi = blocksMap.getStoredBlock(b); + BlockInfoContiguous bi = getStoredBlock(b); if (bi == null) { if (LOG.isDebugEnabled()) { LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " + @@ -2091,7 +2091,7 @@ private void processFirstBlockReport( continue; } - BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(iblk); + BlockInfoContiguous storedBlock = getStoredBlock(iblk); // If block does not belong to any file, we are done. if (storedBlock == null) continue; @@ -2231,7 +2231,7 @@ private BlockInfoContiguous processReportedBlock( } // find block by blockId - BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(block); + BlockInfoContiguous storedBlock = getStoredBlock(block); if(storedBlock == null) { // If blocksMap does not contain reported block id, // the replica should be removed from the data-node. @@ -2529,7 +2529,7 @@ private Block addStoredBlock(final BlockInfoContiguous block, DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); if (block instanceof BlockInfoContiguousUnderConstruction) { //refresh our copy in case the block got completed in another thread - storedBlock = blocksMap.getStoredBlock(block); + storedBlock = getStoredBlock(block); } else { storedBlock = block; } @@ -3410,7 +3410,15 @@ public void removeBlock(Block block) { } public BlockInfoContiguous getStoredBlock(Block block) { - return blocksMap.getStoredBlock(block); + BlockInfoContiguous info = null; + if (BlockIdManager.isStripedBlockID(block.getBlockId())) { + info = blocksMap.getStoredBlock( + new Block(BlockIdManager.convertToGroupID(block.getBlockId()))); + } + if (info == null) { + info = blocksMap.getStoredBlock(block); + } + return info; } /** updates a block in under replication queue */ From ec480134262fe72a8687a25f9a06dd1e8a39b164 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Sun, 8 Feb 2015 16:01:03 -0800 Subject: [PATCH 004/212] Fix Compilation Error in TestAddBlockgroup.java after the merge --- .../apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java index 95133ce77c234..06dfade3f4f0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -75,7 +75,7 @@ public void testAddBlockGroup() throws Exception { final Path file1 = new Path("/file1"); DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L); INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile(); - BlockInfo[] file1Blocks = file1Node.getBlocks(); + BlockInfoContiguous[] file1Blocks = file1Node.getBlocks(); assertEquals(2, file1Blocks.length); assertEquals(GROUP_SIZE, file1Blocks[0].numNodes()); assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP, From e50bcea83d5f6b02ab03b06a3fbf1ed6b8ff4871 Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Thu, 29 Jan 2015 14:15:13 +0530 Subject: [PATCH 005/212] HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding (Kai Zheng via umamahesh) --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 4 + .../apache/hadoop/io/erasurecode/ECChunk.java | 82 ++++++++++++++++ .../rawcoder/AbstractRawErasureCoder.java | 63 +++++++++++++ .../rawcoder/AbstractRawErasureDecoder.java | 93 +++++++++++++++++++ .../rawcoder/AbstractRawErasureEncoder.java | 93 +++++++++++++++++++ .../erasurecode/rawcoder/RawErasureCoder.java | 78 ++++++++++++++++ .../rawcoder/RawErasureDecoder.java | 55 +++++++++++ .../rawcoder/RawErasureEncoder.java | 54 +++++++++++ 8 files changed, 522 insertions(+) create mode 100644 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt new file mode 100644 index 0000000000000..8ce5a894ab721 --- /dev/null +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -0,0 +1,4 @@ + BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285) + + HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding + (Kai Zheng via umamahesh) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java new file mode 100644 index 0000000000000..f84eb11f2d441 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -0,0 +1,82 @@ +/** + * 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.io.erasurecode; + +import java.nio.ByteBuffer; + +/** + * A wrapper for ByteBuffer or bytes array for an erasure code chunk. + */ +public class ECChunk { + + private ByteBuffer chunkBuffer; + + /** + * Wrapping a ByteBuffer + * @param buffer + */ + public ECChunk(ByteBuffer buffer) { + this.chunkBuffer = buffer; + } + + /** + * Wrapping a bytes array + * @param buffer + */ + public ECChunk(byte[] buffer) { + this.chunkBuffer = ByteBuffer.wrap(buffer); + } + + /** + * Convert to ByteBuffer + * @return ByteBuffer + */ + public ByteBuffer getBuffer() { + return chunkBuffer; + } + + /** + * Convert an array of this chunks to an array of ByteBuffers + * @param chunks + * @return an array of ByteBuffers + */ + public static ByteBuffer[] toBuffers(ECChunk[] chunks) { + ByteBuffer[] buffers = new ByteBuffer[chunks.length]; + + for (int i = 0; i < chunks.length; i++) { + buffers[i] = chunks[i].getBuffer(); + } + + return buffers; + } + + /** + * Convert an array of this chunks to an array of byte array + * @param chunks + * @return an array of byte array + */ + public static byte[][] toArray(ECChunk[] chunks) { + byte[][] bytesArr = new byte[chunks.length][]; + + for (int i = 0; i < chunks.length; i++) { + bytesArr[i] = chunks[i].getBuffer().array(); + } + + return bytesArr; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java new file mode 100644 index 0000000000000..474542b8f9a4b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -0,0 +1,63 @@ +/** + * 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.io.erasurecode.rawcoder; + +/** + * A common class of basic facilities to be shared by encoder and decoder + * + * It implements the {@link RawErasureCoder} interface. + */ +public abstract class AbstractRawErasureCoder implements RawErasureCoder { + + private int dataSize; + private int paritySize; + private int chunkSize; + + @Override + public void initialize(int numDataUnits, int numParityUnits, + int chunkSize) { + this.dataSize = numDataUnits; + this.paritySize = numParityUnits; + this.chunkSize = chunkSize; + } + + @Override + public int getNumDataUnits() { + return dataSize; + } + + @Override + public int getNumParityUnits() { + return paritySize; + } + + @Override + public int getChunkSize() { + return chunkSize; + } + + @Override + public boolean preferNativeBuffer() { + return false; + } + + @Override + public void release() { + // Nothing to do by default + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java new file mode 100644 index 0000000000000..4613b25eb13f3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -0,0 +1,93 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; + +/** + * An abstract raw erasure decoder that's to be inherited by new decoders. + * + * It implements the {@link RawErasureDecoder} interface. + */ +public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder + implements RawErasureDecoder { + + @Override + public void decode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + if (erasedIndexes.length == 0) { + return; + } + + doDecode(inputs, erasedIndexes, outputs); + } + + /** + * Perform the real decoding using ByteBuffer + * @param inputs + * @param erasedIndexes + * @param outputs + */ + protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs); + + @Override + public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) { + if (erasedIndexes.length == 0) { + return; + } + + doDecode(inputs, erasedIndexes, outputs); + } + + /** + * Perform the real decoding using bytes array + * @param inputs + * @param erasedIndexes + * @param outputs + */ + protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes, + byte[][] outputs); + + @Override + public void decode(ECChunk[] inputs, int[] erasedIndexes, + ECChunk[] outputs) { + doDecode(inputs, erasedIndexes, outputs); + } + + /** + * Perform the real decoding using chunks + * @param inputs + * @param erasedIndexes + * @param outputs + */ + protected void doDecode(ECChunk[] inputs, int[] erasedIndexes, + ECChunk[] outputs) { + if (inputs[0].getBuffer().hasArray()) { + byte[][] inputBytesArr = ECChunk.toArray(inputs); + byte[][] outputBytesArr = ECChunk.toArray(outputs); + doDecode(inputBytesArr, erasedIndexes, outputBytesArr); + } else { + ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs); + ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs); + doDecode(inputBuffers, erasedIndexes, outputBuffers); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java new file mode 100644 index 0000000000000..4feaf39f28302 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -0,0 +1,93 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; + +/** + * An abstract raw erasure encoder that's to be inherited by new encoders. + * + * It implements the {@link RawErasureEncoder} interface. + */ +public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder + implements RawErasureEncoder { + + @Override + public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { + assert (inputs.length == getNumDataUnits()); + assert (outputs.length == getNumParityUnits()); + + doEncode(inputs, outputs); + } + + /** + * Perform the real encoding work using ByteBuffer + * @param inputs + * @param outputs + */ + protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs); + + @Override + public void encode(byte[][] inputs, byte[][] outputs) { + assert (inputs.length == getNumDataUnits()); + assert (outputs.length == getNumParityUnits()); + + doEncode(inputs, outputs); + } + + /** + * Perform the real encoding work using bytes array + * @param inputs + * @param outputs + */ + protected abstract void doEncode(byte[][] inputs, byte[][] outputs); + + @Override + public void encode(ECChunk[] inputs, ECChunk[] outputs) { + assert (inputs.length == getNumDataUnits()); + assert (outputs.length == getNumParityUnits()); + + doEncode(inputs, outputs); + } + + /** + * Perform the real encoding work using chunks. + * @param inputs + * @param outputs + */ + protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) { + /** + * Note callers may pass byte array, or ByteBuffer via ECChunk according + * to how ECChunk is created. Some implementations of coder use byte array + * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the + * better performance. + */ + if (inputs[0].getBuffer().hasArray()) { + byte[][] inputBytesArr = ECChunk.toArray(inputs); + byte[][] outputBytesArr = ECChunk.toArray(outputs); + doEncode(inputBytesArr, outputBytesArr); + } else { + ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs); + ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs); + doEncode(inputBuffers, outputBuffers); + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java new file mode 100644 index 0000000000000..91a9abfe2aad4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -0,0 +1,78 @@ +/** + * 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.io.erasurecode.rawcoder; + +/** + * RawErasureCoder is a common interface for {@link RawErasureEncoder} and + * {@link RawErasureDecoder} as both encoder and decoder share some properties. + * + * RawErasureCoder is part of ErasureCodec framework, where ErasureCoder is + * used to encode/decode a group of blocks (BlockGroup) according to the codec + * specific BlockGroup layout and logic. An ErasureCoder extracts chunks of + * data from the blocks and can employ various low level RawErasureCoders to + * perform encoding/decoding against the chunks. + * + * To distinguish from ErasureCoder, here RawErasureCoder is used to mean the + * low level constructs, since it only takes care of the math calculation with + * a group of byte buffers. + */ +public interface RawErasureCoder { + + /** + * Initialize with the important parameters for the code. + * @param numDataUnits how many data inputs for the coding + * @param numParityUnits how many parity outputs the coding generates + * @param chunkSize the size of the input/output buffer + */ + public void initialize(int numDataUnits, int numParityUnits, int chunkSize); + + /** + * The number of data input units for the coding. A unit can be a byte, + * chunk or buffer or even a block. + * @return count of data input units + */ + public int getNumDataUnits(); + + /** + * The number of parity output units for the coding. A unit can be a byte, + * chunk, buffer or even a block. + * @return count of parity output units + */ + public int getNumParityUnits(); + + /** + * Chunk buffer size for the input/output + * @return chunk buffer size + */ + public int getChunkSize(); + + /** + * Tell if native or off-heap buffer is preferred or not. It's for callers to + * decide how to allocate coding chunk buffers, either on heap or off heap. + * It will return false by default. + * @return true if native buffer is preferred for performance consideration, + * otherwise false. + */ + public boolean preferNativeBuffer(); + + /** + * Should be called when release this coder. Good chance to release encoding + * or decoding buffers + */ + public void release(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java new file mode 100644 index 0000000000000..1358b7d0bc534 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java @@ -0,0 +1,55 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; + +/** + * RawErasureDecoder performs decoding given chunks of input data and generates + * missing data that corresponds to an erasure code scheme, like XOR and + * Reed-Solomon. + * + * It extends the {@link RawErasureCoder} interface. + */ +public interface RawErasureDecoder extends RawErasureCoder { + + /** + * Decode with inputs and erasedIndexes, generates outputs + * @param inputs + * @param outputs + */ + public void decode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs); + + /** + * Decode with inputs and erasedIndexes, generates outputs + * @param inputs + * @param outputs + */ + public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs); + + /** + * Decode with inputs and erasedIndexes, generates outputs + * @param inputs + * @param outputs + */ + public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java new file mode 100644 index 0000000000000..974f86ca6ca62 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java @@ -0,0 +1,54 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; + +/** + * RawErasureEncoder performs encoding given chunks of input data and generates + * parity outputs that corresponds to an erasure code scheme, like XOR and + * Reed-Solomon. + * + * It extends the {@link RawErasureCoder} interface. + */ +public interface RawErasureEncoder extends RawErasureCoder { + + /** + * Encode with inputs and generates outputs + * @param inputs + * @param outputs + */ + public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs); + + /** + * Encode with inputs and generates outputs + * @param inputs + * @param outputs + */ + public void encode(byte[][] inputs, byte[][] outputs); + + /** + * Encode with inputs and generates outputs + * @param inputs + * @param outputs + */ + public void encode(ECChunk[] inputs, ECChunk[] outputs); + +} From 9f19eb9fcf47c703c7b9ed7be573266d18baa051 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 2 Feb 2015 14:39:53 +0530 Subject: [PATCH 006/212] HADOOP-11534. Minor improvements for raw erasure coders ( Contributed by Kai Zheng ) --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 5 ++++- .../org/apache/hadoop/io/erasurecode/ECChunk.java | 15 +++++++++++++-- .../rawcoder/AbstractRawErasureCoder.java | 12 ++++++------ 3 files changed, 23 insertions(+), 9 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 8ce5a894ab721..2124800e6d3fe 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -1,4 +1,7 @@ BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285) HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding - (Kai Zheng via umamahesh) \ No newline at end of file + (Kai Zheng via umamahesh) + + HADOOP-11534. Minor improvements for raw erasure coders + ( Kai Zheng via vinayakumarb ) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java index f84eb11f2d441..01e8f3580e505 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -66,15 +66,26 @@ public static ByteBuffer[] toBuffers(ECChunk[] chunks) { } /** - * Convert an array of this chunks to an array of byte array + * Convert an array of this chunks to an array of byte array. + * Note the chunk buffers are not affected. * @param chunks * @return an array of byte array */ public static byte[][] toArray(ECChunk[] chunks) { byte[][] bytesArr = new byte[chunks.length][]; + ByteBuffer buffer; for (int i = 0; i < chunks.length; i++) { - bytesArr[i] = chunks[i].getBuffer().array(); + buffer = chunks[i].getBuffer(); + if (buffer.hasArray()) { + bytesArr[i] = buffer.array(); + } else { + bytesArr[i] = new byte[buffer.remaining()]; + // Avoid affecting the original one + buffer.mark(); + buffer.get(bytesArr[i]); + buffer.reset(); + } } return bytesArr; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index 474542b8f9a4b..74d2ab6fc1ecf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -24,26 +24,26 @@ */ public abstract class AbstractRawErasureCoder implements RawErasureCoder { - private int dataSize; - private int paritySize; + private int numDataUnits; + private int numParityUnits; private int chunkSize; @Override public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { - this.dataSize = numDataUnits; - this.paritySize = numParityUnits; + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; this.chunkSize = chunkSize; } @Override public int getNumDataUnits() { - return dataSize; + return numDataUnits; } @Override public int getNumParityUnits() { - return paritySize; + return numParityUnits; } @Override From b29f3bde4d2fd2f2c4abd6d7b5f97a81bb50efb2 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Sun, 8 Feb 2015 01:40:27 +0800 Subject: [PATCH 007/212] HADOOP-11541. Raw XOR coder --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 5 +- .../erasurecode/rawcoder/XorRawDecoder.java | 81 ++++++ .../erasurecode/rawcoder/XorRawEncoder.java | 61 ++++ .../hadoop/io/erasurecode/TestCoderBase.java | 262 ++++++++++++++++++ .../rawcoder/TestRawCoderBase.java | 96 +++++++ .../erasurecode/rawcoder/TestXorRawCoder.java | 52 ++++ 6 files changed, 556 insertions(+), 1 deletion(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 2124800e6d3fe..9728f977bba70 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -4,4 +4,7 @@ (Kai Zheng via umamahesh) HADOOP-11534. Minor improvements for raw erasure coders - ( Kai Zheng via vinayakumarb ) \ No newline at end of file + ( Kai Zheng via vinayakumarb ) + + HADOOP-11541. Raw XOR coder + ( Kai Zheng ) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java new file mode 100644 index 0000000000000..98307a7b3c2f6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java @@ -0,0 +1,81 @@ +/** + * 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.io.erasurecode.rawcoder; + +import java.nio.ByteBuffer; + +/** + * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID. + */ +public class XorRawDecoder extends AbstractRawErasureDecoder { + + @Override + protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + assert(erasedIndexes.length == outputs.length); + assert(erasedIndexes.length <= 1); + + int bufSize = inputs[0].remaining(); + int erasedIdx = erasedIndexes[0]; + + // Set the output to zeros. + for (int j = 0; j < bufSize; j++) { + outputs[0].put(j, (byte) 0); + } + + // Process the inputs. + for (int i = 0; i < inputs.length; i++) { + // Skip the erased location. + if (i == erasedIdx) { + continue; + } + + for (int j = 0; j < bufSize; j++) { + outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j))); + } + } + } + + @Override + protected void doDecode(byte[][] inputs, int[] erasedIndexes, + byte[][] outputs) { + assert(erasedIndexes.length == outputs.length); + assert(erasedIndexes.length <= 1); + + int bufSize = inputs[0].length; + int erasedIdx = erasedIndexes[0]; + + // Set the output to zeros. + for (int j = 0; j < bufSize; j++) { + outputs[0][j] = 0; + } + + // Process the inputs. + for (int i = 0; i < inputs.length; i++) { + // Skip the erased location. + if (i == erasedIdx) { + continue; + } + + for (int j = 0; j < bufSize; j++) { + outputs[0][j] ^= inputs[i][j]; + } + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java new file mode 100644 index 0000000000000..99b20b92e7ad4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java @@ -0,0 +1,61 @@ +/** + * 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.io.erasurecode.rawcoder; + +import java.nio.ByteBuffer; + +/** + * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID. + */ +public class XorRawEncoder extends AbstractRawErasureEncoder { + + @Override + protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { + int bufSize = inputs[0].remaining(); + + // Get the first buffer's data. + for (int j = 0; j < bufSize; j++) { + outputs[0].put(j, inputs[0].get(j)); + } + + // XOR with everything else. + for (int i = 1; i < inputs.length; i++) { + for (int j = 0; j < bufSize; j++) { + outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j))); + } + } + } + + @Override + protected void doEncode(byte[][] inputs, byte[][] outputs) { + int bufSize = inputs[0].length; + + // Get the first buffer's data. + for (int j = 0; j < bufSize; j++) { + outputs[0][j] = inputs[0][j]; + } + + // XOR with everything else. + for (int i = 1; i < inputs.length; i++) { + for (int j = 0; j < bufSize; j++) { + outputs[0][j] ^= inputs[i][j]; + } + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java new file mode 100644 index 0000000000000..9482b4348254c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -0,0 +1,262 @@ +/** + * 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.io.erasurecode; + +import java.nio.ByteBuffer; +import java.util.Random; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Test base of common utilities for tests not only raw coders but also block + * coders. + */ +public abstract class TestCoderBase { + protected static Random RAND = new Random(); + + protected int numDataUnits; + protected int numParityUnits; + protected int chunkSize = 16 * 1024; + + // Indexes of erased data units. Will also support test of erasing + // parity units + protected int[] erasedDataIndexes = new int[] {0}; + + // Data buffers are either direct or on-heap, for performance the two cases + // may go to different coding implementations. + protected boolean usingDirectBuffer = true; + + /** + * Compare and verify if erased chunks are equal to recovered chunks + * @param erasedChunks + * @param recoveredChunks + */ + protected void compareAndVerify(ECChunk[] erasedChunks, + ECChunk[] recoveredChunks) { + byte[][] erased = ECChunk.toArray(erasedChunks); + byte[][] recovered = ECChunk.toArray(recoveredChunks); + for (int i = 0; i < erasedChunks.length; ++i) { + assertArrayEquals("Decoding and comparing failed.", erased[i], + recovered[i]); + } + } + + /** + * Adjust and return erased indexes based on the array of the input chunks ( + * parity chunks + data chunks). + * @return + */ + protected int[] getErasedIndexesForDecoding() { + int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length]; + for (int i = 0; i < erasedDataIndexes.length; ++i) { + erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits; + } + return erasedIndexesForDecoding; + } + + /** + * Return input chunks for decoding, which is parityChunks + dataChunks. + * @param dataChunks + * @param parityChunks + * @return + */ + protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks, + ECChunk[] parityChunks) { + ECChunk[] inputChunks = new ECChunk[numParityUnits + numDataUnits]; + + int idx = 0; + for (int i = 0; i < numParityUnits; i++) { + inputChunks[idx ++] = parityChunks[i]; + } + for (int i = 0; i < numDataUnits; i++) { + inputChunks[idx ++] = dataChunks[i]; + } + + return inputChunks; + } + + /** + * Have a copy of the data chunks that's to be erased thereafter. The copy + * will be used to compare and verify with the to be recovered chunks. + * @param dataChunks + * @return + */ + protected ECChunk[] copyDataChunksToErase(ECChunk[] dataChunks) { + ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length]; + + int j = 0; + for (int i = 0; i < erasedDataIndexes.length; ++i) { + copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]); + } + + return copiedChunks; + } + + /** + * Erase some data chunks to test the recovering of them + * @param dataChunks + */ + protected void eraseSomeDataBlocks(ECChunk[] dataChunks) { + for (int i = 0; i < erasedDataIndexes.length; ++i) { + eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]); + } + } + + /** + * Erase data from the specified chunks, putting ZERO bytes to the buffers. + * @param chunks + */ + protected void eraseDataFromChunks(ECChunk[] chunks) { + for (int i = 0; i < chunks.length; ++i) { + eraseDataFromChunk(chunks[i]); + } + } + + /** + * Erase data from the specified chunk, putting ZERO bytes to the buffer. + * @param chunk + */ + protected void eraseDataFromChunk(ECChunk chunk) { + ByteBuffer chunkBuffer = chunk.getBuffer(); + // erase the data + chunkBuffer.position(0); + for (int i = 0; i < chunkSize; ++i) { + chunkBuffer.put((byte) 0); + } + chunkBuffer.flip(); + } + + /** + * Clone chunks along with copying the associated data. It respects how the + * chunk buffer is allocated, direct or non-direct. It avoids affecting the + * original chunk buffers. + * @param chunks + * @return + */ + protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) { + ECChunk[] results = new ECChunk[chunks.length]; + for (int i = 0; i < chunks.length; ++i) { + results[i] = cloneChunkWithData(chunks[i]); + } + + return results; + } + + /** + * Clone chunk along with copying the associated data. It respects how the + * chunk buffer is allocated, direct or non-direct. It avoids affecting the + * original chunk. + * @param chunk + * @return a new chunk + */ + protected static ECChunk cloneChunkWithData(ECChunk chunk) { + ByteBuffer srcBuffer = chunk.getBuffer(); + ByteBuffer destBuffer; + + byte[] bytesArr = new byte[srcBuffer.remaining()]; + srcBuffer.mark(); + srcBuffer.get(bytesArr); + srcBuffer.reset(); + + if (srcBuffer.hasArray()) { + destBuffer = ByteBuffer.wrap(bytesArr); + } else { + destBuffer = ByteBuffer.allocateDirect(srcBuffer.remaining()); + destBuffer.put(bytesArr); + destBuffer.flip(); + } + + return new ECChunk(destBuffer); + } + + /** + * Allocate a chunk for output or writing. + * @return + */ + protected ECChunk allocateOutputChunk() { + ByteBuffer buffer = allocateOutputBuffer(); + + return new ECChunk(buffer); + } + + /** + * Allocate a buffer for output or writing. + * @return + */ + protected ByteBuffer allocateOutputBuffer() { + ByteBuffer buffer = usingDirectBuffer ? + ByteBuffer.allocateDirect(chunkSize) : ByteBuffer.allocate(chunkSize); + + return buffer; + } + + /** + * Prepare data chunks for each data unit, by generating random data. + * @return + */ + protected ECChunk[] prepareDataChunksForEncoding() { + ECChunk[] chunks = new ECChunk[numDataUnits]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = generateDataChunk(); + } + + return chunks; + } + + /** + * Generate data chunk by making random data. + * @return + */ + protected ECChunk generateDataChunk() { + ByteBuffer buffer = allocateOutputBuffer(); + for (int i = 0; i < chunkSize; i++) { + buffer.put((byte) RAND.nextInt(256)); + } + buffer.flip(); + + return new ECChunk(buffer); + } + + /** + * Prepare parity chunks for encoding, each chunk for each parity unit. + * @return + */ + protected ECChunk[] prepareParityChunksForEncoding() { + ECChunk[] chunks = new ECChunk[numParityUnits]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = allocateOutputChunk(); + } + + return chunks; + } + + /** + * Prepare output chunks for decoding, each output chunk for each erased + * chunk. + * @return + */ + protected ECChunk[] prepareOutputChunksForDecoding() { + ECChunk[] chunks = new ECChunk[erasedDataIndexes.length]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = allocateOutputChunk(); + } + + return chunks; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java new file mode 100644 index 0000000000000..9119211641fa4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -0,0 +1,96 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; +import org.apache.hadoop.io.erasurecode.TestCoderBase; + +/** + * Raw coder test base with utilities. + */ +public abstract class TestRawCoderBase extends TestCoderBase { + protected Class encoderClass; + protected Class decoderClass; + + /** + * Generating source data, encoding, recovering and then verifying. + * RawErasureCoder mainly uses ECChunk to pass input and output data buffers, + * it supports two kinds of ByteBuffers, one is array backed, the other is + * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test. + * @param usingDirectBuffer + */ + protected void testCoding(boolean usingDirectBuffer) { + // Generate data and encode + ECChunk[] dataChunks = prepareDataChunksForEncoding(); + ECChunk[] parityChunks = prepareParityChunksForEncoding(); + RawErasureEncoder encoder = createEncoder(); + + // Backup all the source chunks for later recovering because some coders + // may affect the source data. + ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks); + // Make a copy of a strip for later comparing + ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks); + + encoder.encode(dataChunks, parityChunks); + // Erase the copied sources + eraseSomeDataBlocks(clonedDataChunks); + + //Decode + ECChunk[] inputChunks = prepareInputChunksForDecoding(clonedDataChunks, + parityChunks); + ECChunk[] recoveredChunks = prepareOutputChunksForDecoding(); + RawErasureDecoder decoder = createDecoder(); + decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks); + + //Compare + compareAndVerify(toEraseDataChunks, recoveredChunks); + } + + /** + * Create the raw erasure encoder to test + * @return + */ + protected RawErasureEncoder createEncoder() { + RawErasureEncoder encoder; + try { + encoder = encoderClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed to create encoder", e); + } + + encoder.initialize(numDataUnits, numParityUnits, chunkSize); + return encoder; + } + + /** + * create the raw erasure decoder to test + * @return + */ + protected RawErasureDecoder createDecoder() { + RawErasureDecoder decoder; + try { + decoder = decoderClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed to create decoder", e); + } + + decoder.initialize(numDataUnits, numParityUnits, chunkSize); + return decoder; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java new file mode 100644 index 0000000000000..8e59b8a2c3ca7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java @@ -0,0 +1,52 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Random; + +/** + * Test XOR encoding and decoding. + */ +public class TestXorRawCoder extends TestRawCoderBase { + private static Random RAND = new Random(); + + @Before + public void setup() { + this.encoderClass = XorRawEncoder.class; + this.decoderClass = XorRawDecoder.class; + + this.numDataUnits = 10; + this.numParityUnits = 1; + + this.erasedDataIndexes = new int[] {0}; + } + + @Test + public void testCodingNoDirectBuffer() { + testCoding(false); + } + + @Test + public void testCodingDirectBuffer() { + testCoding(true); + } + +} From ba9371492036983a9899398907ab41fe548f29b3 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 10 Feb 2015 17:54:10 -0800 Subject: [PATCH 008/212] HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info. Contributed by Jing Zhao. --- .../blockmanagement/BlockCollection.java | 13 +- .../blockmanagement/BlockIdManager.java | 7 +- .../server/blockmanagement/BlockInfo.java | 343 ++++++++++++++++++ .../blockmanagement/BlockInfoContiguous.java | 262 +++---------- .../BlockInfoContiguousUnderConstruction.java | 137 +------ .../blockmanagement/BlockInfoStriped.java | 179 +++++++++ .../server/blockmanagement/BlockManager.java | 188 +++++----- .../server/blockmanagement/BlocksMap.java | 46 +-- .../CacheReplicationMonitor.java | 10 +- .../blockmanagement/DatanodeDescriptor.java | 22 +- .../blockmanagement/DatanodeStorageInfo.java | 38 +- .../ReplicaUnderConstruction.java | 119 ++++++ .../hdfs/server/namenode/FSNamesystem.java | 19 +- .../hdfs/server/namenode/NamenodeFsck.java | 3 +- .../snapshot/FSImageFormatPBSnapshot.java | 4 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 4 +- .../server/blockmanagement/TestBlockInfo.java | 6 +- .../blockmanagement/TestBlockInfoStriped.java | 219 +++++++++++ .../blockmanagement/TestBlockManager.java | 4 +- .../TestReplicationPolicy.java | 2 +- 20 files changed, 1122 insertions(+), 503 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java index c0a959c926824..e873946a52006 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java @@ -39,12 +39,12 @@ public interface BlockCollection { public ContentSummary computeContentSummary(BlockStoragePolicySuite bsps); /** - * @return the number of blocks + * @return the number of blocks or block groups */ public int numBlocks(); /** - * Get the blocks. + * Get the blocks or block groups. */ public BlockInfoContiguous[] getBlocks(); @@ -55,8 +55,8 @@ public interface BlockCollection { public long getPreferredBlockSize(); /** - * Get block replication for the collection - * @return block replication value + * Get block replication for the collection. + * @return block replication value. Return 0 if the file is erasure coded. */ public short getPreferredBlockReplication(); @@ -71,7 +71,7 @@ public interface BlockCollection { public String getName(); /** - * Set the block at the given index. + * Set the block/block-group at the given index. */ public void setBlock(int index, BlockInfoContiguous blk); @@ -79,7 +79,8 @@ public interface BlockCollection { * Convert the last block of the collection to an under-construction block * and set the locations. */ - public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfoContiguous lastBlock, + public BlockInfoContiguousUnderConstruction setLastBlock( + BlockInfoContiguous lastBlock, DatanodeStorageInfo[] targets) throws IOException; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index 230d9a5d1e937..fa800c572f25c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -218,6 +218,11 @@ public static boolean isStripedBlockID(long id) { } public static long convertToGroupID(long id) { - return id & (~(HdfsConstants.MAX_BLOCKS_IN_GROUP - 1)); + return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK); + } + + public static int getBlockIndex(Block reportedBlock) { + return (int) (reportedBlock.getBlockId() & + HdfsConstants.BLOCK_GROUP_INDEX_MASK); } } 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 new file mode 100644 index 0000000000000..d3ea813b2eb63 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java @@ -0,0 +1,343 @@ +/** + * 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 org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.util.LightWeightGSet; + +import java.util.LinkedList; + +/** + * For a given block (or an erasure coding block group), BlockInfo class + * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes + * where the replicas of the block, or blocks belonging to the erasure coding + * block group, are stored. + */ +public abstract class BlockInfo extends Block + implements LightWeightGSet.LinkedElement { + private BlockCollection bc; + + /** For implementing {@link LightWeightGSet.LinkedElement} interface */ + private LightWeightGSet.LinkedElement nextLinkedElement; + + /** + * 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 + * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16 + * bytes using the triplets. + */ + protected Object[] triplets; + + /** + * Construct an entry for blocksmap + * @param size the block's replication factor, or the total number of blocks + * in the block group + */ + public BlockInfo(short size) { + this.triplets = new Object[3 * size]; + this.bc = null; + } + + public BlockInfo(Block blk, short size) { + super(blk); + this.triplets = new Object[3 * size]; + this.bc = null; + } + + public BlockCollection getBlockCollection() { + return bc; + } + + public void setBlockCollection(BlockCollection bc) { + this.bc = bc; + } + + 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 (DatanodeStorageInfo)triplets[index*3]; + } + + BlockInfo getPrevious(int index) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; + return (BlockInfo) triplets[index*3+1]; + } + + BlockInfo getNext(int index) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; + return (BlockInfo) triplets[index*3+2]; + } + + void setStorageInfo(int index, DatanodeStorageInfo storage) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3 < triplets.length : "Index is out of bound"; + triplets[index*3] = storage; + } + + /** + * Return the previous block on the block list for the datanode at + * position index. Set the previous block on the list to "to". + * + * @param index - the datanode index + * @param to - block to be set to previous on the list of blocks + * @return current previous block on the list of blocks + */ + BlockInfo setPrevious(int index, BlockInfo to) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; + BlockInfo info = (BlockInfo) triplets[index*3+1]; + triplets[index*3+1] = to; + return info; + } + + /** + * Return the next block on the block list for the datanode at + * position index. Set the next block on the list to "to". + * + * @param index - the datanode index + * @param to - block to be set to next on the list of blocks + * @return current next block on the list of blocks + */ + BlockInfo setNext(int index, BlockInfo to) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; + BlockInfo info = (BlockInfo) triplets[index*3+2]; + triplets[index*3+2] = to; + return info; + } + + public int getCapacity() { + assert this.triplets != null : "BlockInfo is not initialized"; + assert triplets.length % 3 == 0 : "Malformed BlockInfo"; + return triplets.length / 3; + } + + /** + * Count the number of data-nodes the block currently belongs to (i.e., NN + * has received block reports from the DN). + */ + public abstract int numNodes(); + + /** + * Add a {@link DatanodeStorageInfo} location for a block + * @param storage The storage to add + * @param reportedBlock The block reported from the datanode. This is only + * used by erasure coded blocks, this block's id contains + * information indicating the index of the block in the + * corresponding block group. + */ + abstract boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock); + + /** + * Remove {@link DatanodeStorageInfo} location for a block + */ + abstract boolean removeStorage(DatanodeStorageInfo storage); + + /** + * Replace the current BlockInfo with the new one in corresponding + * DatanodeStorageInfo's linked list + */ + abstract void replaceBlock(BlockInfo newBlock); + + /** + * Find specified DatanodeDescriptor. + * @return index or -1 if not found. + */ + boolean findDatanode(DatanodeDescriptor dn) { + int len = getCapacity(); + for (int idx = 0; idx < len; idx++) { + DatanodeDescriptor cur = getDatanode(idx); + if(cur == dn) { + return true; + } + } + return false; + } + + /** + * Find specified DatanodeStorageInfo. + * @return DatanodeStorageInfo or null if not found. + */ + DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) { + int len = getCapacity(); + for(int idx = 0; idx < len; idx++) { + DatanodeStorageInfo cur = getStorageInfo(idx); + if(cur != null && cur.getDatanodeDescriptor() == dn) { + return cur; + } + } + return null; + } + + /** + * Find specified DatanodeStorageInfo. + * @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; + } + } + return -1; + } + + /** + * Insert this block into the head of the list of blocks + * 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. + */ + 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.findStorageInfo(storage), this); + } + return this; + } + + /** + * Remove this block from the list of blocks + * 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 + * empy after deletion. + */ + BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) { + if (head == null) { + return null; + } + int dnIndex = this.findStorageInfo(storage); + if (dnIndex < 0) { // this block is not on the data-node list + return head; + } + + BlockInfo next = this.getNext(dnIndex); + BlockInfo prev = this.getPrevious(dnIndex); + this.setNext(dnIndex, null); + this.setPrevious(dnIndex, null); + if (prev != null) { + prev.setNext(prev.findStorageInfo(storage), next); + } + if (next != null) { + next.setPrevious(next.findStorageInfo(storage), prev); + } + if (this == head) { // removing the head + head = next; + } + return head; + } + + /** + * Remove this block from the list of blocks related to the specified + * DatanodeDescriptor. Insert it into the head of the list of blocks. + * + * @return the new head of the list. + */ + public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage, + int curIndex, int headIndex) { + if (head == this) { + return this; + } + BlockInfo next = this.setNext(curIndex, head); + BlockInfo prev = this.setPrevious(curIndex, null); + + head.setPrevious(headIndex, this); + prev.setNext(prev.findStorageInfo(storage), next); + if (next != null) { + next.setPrevious(next.findStorageInfo(storage), prev); + } + return this; + } + + /** + * BlockInfo represents a block that is not being constructed. + * In order to start modifying the block, the BlockInfo should be converted + * to {@link BlockInfoContiguousUnderConstruction}. + * @return {@link HdfsServerConstants.BlockUCState#COMPLETE} + */ + public HdfsServerConstants.BlockUCState getBlockUCState() { + return HdfsServerConstants.BlockUCState.COMPLETE; + } + + /** + * Is this block complete? + * + * @return true if the state of the block is + * {@link HdfsServerConstants.BlockUCState#COMPLETE} + */ + public boolean isComplete() { + return getBlockUCState().equals(HdfsServerConstants.BlockUCState.COMPLETE); + } + + public boolean isDeleted() { + return (bc == null); + } + + @Override + public int hashCode() { + // Super implementation is sufficient + return super.hashCode(); + } + + @Override + public boolean equals(Object obj) { + // Sufficient to rely on super's implementation + return (this == obj) || super.equals(obj); + } + + @Override + public LightWeightGSet.LinkedElement getNext() { + return nextLinkedElement; + } + + @Override + public void setNext(LightWeightGSet.LinkedElement next) { + this.nextLinkedElement = next; + } + + static BlockInfo copyOf(BlockInfo b) { + if (b instanceof BlockInfoContiguous) { + return new BlockInfoContiguous((BlockInfoContiguous) b); + } else { + return new BlockInfoStriped((BlockInfoStriped) b); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index 769046b1f5c98..e30e022eaa54e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -17,66 +17,34 @@ */ package org.apache.hadoop.hdfs.server.blockmanagement; -import java.util.LinkedList; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; -import org.apache.hadoop.util.LightWeightGSet; /** - * BlockInfo class maintains for a given block - * the {@link BlockCollection} it is part of and datanodes where the replicas of - * the block are stored. + * Subclass of {@link BlockInfo}, used for a block with replication scheme. */ @InterfaceAudience.Private -public class BlockInfoContiguous extends Block - implements LightWeightGSet.LinkedElement { +public class BlockInfoContiguous extends BlockInfo { public static final BlockInfoContiguous[] EMPTY_ARRAY = {}; - private BlockCollection bc; - - /** For implementing {@link LightWeightGSet.LinkedElement} interface */ - private LightWeightGSet.LinkedElement nextLinkedElement; - - /** - * 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 - * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16 - * bytes using the triplets. - */ - private Object[] triplets; - - /** - * Construct an entry for blocksmap - * @param replication the block's replication factor - */ - public BlockInfoContiguous(short replication) { - this.triplets = new Object[3*replication]; - this.bc = null; + public BlockInfoContiguous(short size) { + super(size); } - - public BlockInfoContiguous(Block blk, short replication) { - super(blk); - this.triplets = new Object[3*replication]; - this.bc = null; + + public BlockInfoContiguous(Block blk, short size) { + super(blk, size); } /** * Copy construction. - * This is used to convert BlockInfoUnderConstruction - * @param from BlockInfo to copy from. + * This is used to convert BlockReplicationInfoUnderConstruction + * @param from BlockReplicationInfo to copy from. */ protected BlockInfoContiguous(BlockInfoContiguous from) { - super(from); + this(from, from.getBlockCollection().getBlockReplication()); this.triplets = new Object[from.triplets.length]; - this.bc = from.bc; + this.setBlockCollection(from.getBlockCollection()); } public BlockCollection getBlockCollection() { @@ -173,9 +141,10 @@ public int getCapacity() { private int ensureCapacity(int num) { assert this.triplets != null : "BlockInfo is not initialized"; int last = numNodes(); - if(triplets.length >= (last+num)*3) + if (triplets.length >= (last+num)*3) { return last; - /* Not enough space left. Create a new array. Should normally + } + /* Not enough space left. Create a new array. Should normally * happen only when replication is manually increased by the user. */ Object[] old = triplets; triplets = new Object[(last+num)*3]; @@ -183,23 +152,8 @@ private int ensureCapacity(int num) { return last; } - /** - * Count the number of data-nodes the block belongs to. - */ - public int numNodes() { - assert this.triplets != null : "BlockInfo is not initialized"; - assert triplets.length % 3 == 0 : "Malformed BlockInfo"; - for(int idx = getCapacity()-1; idx >= 0; idx--) { - if(getDatanode(idx) != null) - return idx+1; - } - return 0; - } - - /** - * Add a {@link DatanodeStorageInfo} location for a block - */ - boolean addStorage(DatanodeStorageInfo storage) { + @Override + boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) { // find the last null node int lastNode = ensureCapacity(1); setStorageInfo(lastNode, storage); @@ -208,149 +162,53 @@ boolean addStorage(DatanodeStorageInfo storage) { return true; } - /** - * Remove {@link DatanodeStorageInfo} location for a block - */ + @Override boolean removeStorage(DatanodeStorageInfo storage) { int dnIndex = findStorageInfo(storage); - if(dnIndex < 0) // the node is not found + if (dnIndex < 0) { // the node is not found return false; - assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : - "Block is still in the list and must be removed first."; + } + assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : + "Block is still in the list and must be removed first."; // find the last not null node - int lastNode = numNodes()-1; - // replace current node triplet by the lastNode one + int lastNode = numNodes()-1; + // replace current node triplet by the lastNode one setStorageInfo(dnIndex, getStorageInfo(lastNode)); - setNext(dnIndex, getNext(lastNode)); - setPrevious(dnIndex, getPrevious(lastNode)); + setNext(dnIndex, getNext(lastNode)); + setPrevious(dnIndex, getPrevious(lastNode)); // set the last triplet to null setStorageInfo(lastNode, null); - setNext(lastNode, null); - setPrevious(lastNode, null); + setNext(lastNode, null); + setPrevious(lastNode, null); return true; } - /** - * Find specified DatanodeStorageInfo. - * @return DatanodeStorageInfo or null if not found. - */ - DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) { - int len = getCapacity(); - for(int idx = 0; idx < len; idx++) { - DatanodeStorageInfo cur = getStorageInfo(idx); - if(cur == null) - break; - if(cur.getDatanodeDescriptor() == dn) - return cur; - } - return null; - } - - /** - * Find specified DatanodeStorageInfo. - * @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; + @Override + public int numNodes() { + assert this.triplets != null : "BlockInfo is not initialized"; + assert triplets.length % 3 == 0 : "Malformed BlockInfo"; + + for (int idx = getCapacity()-1; idx >= 0; idx--) { + if (getDatanode(idx) != null) { + return idx + 1; } } - return -1; - } - - /** - * Insert this block into the head of the list of blocks - * 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. - */ - BlockInfoContiguous listInsert(BlockInfoContiguous 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.findStorageInfo(storage), this); - return this; - } - - /** - * Remove this block from the list of blocks - * 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 - * empy after deletion. - */ - BlockInfoContiguous listRemove(BlockInfoContiguous head, - DatanodeStorageInfo storage) { - if(head == null) - return null; - int dnIndex = this.findStorageInfo(storage); - if(dnIndex < 0) // this block is not on the data-node list - return head; - - BlockInfoContiguous next = this.getNext(dnIndex); - BlockInfoContiguous prev = this.getPrevious(dnIndex); - this.setNext(dnIndex, null); - this.setPrevious(dnIndex, null); - if(prev != null) - prev.setNext(prev.findStorageInfo(storage), next); - if(next != null) - next.setPrevious(next.findStorageInfo(storage), prev); - if(this == head) // removing the head - head = next; - return head; + return 0; } - /** - * Remove this block from the list of blocks related to the specified - * DatanodeDescriptor. Insert it into the head of the list of blocks. - * - * @return the new head of the list. - */ - public BlockInfoContiguous moveBlockToHead(BlockInfoContiguous head, - DatanodeStorageInfo storage, int curIndex, int headIndex) { - if (head == this) { - return this; - } - BlockInfoContiguous next = this.setNext(curIndex, head); - BlockInfoContiguous prev = this.setPrevious(curIndex, null); - - head.setPrevious(headIndex, this); - prev.setNext(prev.findStorageInfo(storage), next); - if (next != null) { - next.setPrevious(next.findStorageInfo(storage), prev); + @Override + void replaceBlock(BlockInfo newBlock) { + assert newBlock instanceof BlockInfoContiguous; + for (int i = this.numNodes() - 1; i >= 0; i--) { + final DatanodeStorageInfo storage = this.getStorageInfo(i); + final boolean removed = storage.removeBlock(this); + assert removed : "currentBlock not found."; + + final DatanodeStorageInfo.AddBlockResult result = storage.addBlock( + newBlock, newBlock); + assert result == DatanodeStorageInfo.AddBlockResult.ADDED : + "newBlock already exists."; } - return this; - } - - /** - * BlockInfo represents a block that is not being constructed. - * In order to start modifying the block, the BlockInfo should be converted - * to {@link BlockInfoContiguousUnderConstruction}. - * @return {@link BlockUCState#COMPLETE} - */ - public BlockUCState getBlockUCState() { - return BlockUCState.COMPLETE; - } - - /** - * Is this block complete? - * - * @return true if the state of the block is {@link BlockUCState#COMPLETE} - */ - public boolean isComplete() { - return getBlockUCState().equals(BlockUCState.COMPLETE); } /** @@ -368,32 +226,10 @@ public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction( } // the block is already under construction BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction)this; + (BlockInfoContiguousUnderConstruction) this; ucBlock.setBlockUCState(s); ucBlock.setExpectedLocations(targets); ucBlock.setBlockCollection(getBlockCollection()); return ucBlock; } - - @Override - public int hashCode() { - // Super implementation is sufficient - return super.hashCode(); - } - - @Override - public boolean equals(Object obj) { - // Sufficient to rely on super's implementation - return (this == obj) || super.equals(obj); - } - - @Override - public LightWeightGSet.LinkedElement getNext() { - return nextLinkedElement; - } - - @Override - public void setNext(LightWeightGSet.LinkedElement next) { - this.nextLinkedElement = next; - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java index 92153abb97686..c78c9e2c45558 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java @@ -59,101 +59,6 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous { */ private Block truncateBlock; - /** - * ReplicaUnderConstruction contains information about replicas while - * they are under construction. - * The GS, the length and the state of the replica is as reported by - * the data-node. - * It is not guaranteed, but expected, that data-nodes actually have - * corresponding replicas. - */ - static class ReplicaUnderConstruction extends Block { - private final DatanodeStorageInfo expectedLocation; - private ReplicaState state; - private boolean chosenAsPrimary; - - ReplicaUnderConstruction(Block block, - DatanodeStorageInfo target, - ReplicaState state) { - super(block); - this.expectedLocation = target; - this.state = state; - this.chosenAsPrimary = false; - } - - /** - * Expected block replica location as assigned when the block was allocated. - * This defines the pipeline order. - * It is not guaranteed, but expected, that the data-node actually has - * the replica. - */ - private DatanodeStorageInfo getExpectedStorageLocation() { - return expectedLocation; - } - - /** - * Get replica state as reported by the data-node. - */ - ReplicaState getState() { - return state; - } - - /** - * Whether the replica was chosen for recovery. - */ - boolean getChosenAsPrimary() { - return chosenAsPrimary; - } - - /** - * Set replica state. - */ - void setState(ReplicaState s) { - state = s; - } - - /** - * Set whether this replica was chosen for recovery. - */ - void setChosenAsPrimary(boolean chosenAsPrimary) { - this.chosenAsPrimary = chosenAsPrimary; - } - - /** - * Is data-node the replica belongs to alive. - */ - boolean isAlive() { - return expectedLocation.getDatanodeDescriptor().isAlive; - } - - @Override // Block - public int hashCode() { - return super.hashCode(); - } - - @Override // Block - public boolean equals(Object obj) { - // Sufficient to rely on super's implementation - return (this == obj) || super.equals(obj); - } - - @Override - public String toString() { - final StringBuilder b = new StringBuilder(50); - appendStringTo(b); - return b.toString(); - } - - @Override - public void appendStringTo(StringBuilder sb) { - sb.append("ReplicaUC[") - .append(expectedLocation) - .append("|") - .append(state) - .append("]"); - } - } - /** * Create block and set its state to * {@link BlockUCState#UNDER_CONSTRUCTION}. @@ -165,7 +70,8 @@ public BlockInfoContiguousUnderConstruction(Block blk, short replication) { /** * Create a block that is currently being constructed. */ - public BlockInfoContiguousUnderConstruction(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) { + public BlockInfoContiguousUnderConstruction(Block blk, short replication, + BlockUCState state, DatanodeStorageInfo[] targets) { super(blk, replication); assert getBlockUCState() != BlockUCState.COMPLETE : "BlockInfoUnderConstruction cannot be in COMPLETE state"; @@ -191,10 +97,11 @@ assert getBlockUCState() != BlockUCState.COMPLETE : /** Set expected locations */ public void setExpectedLocations(DatanodeStorageInfo[] targets) { int numLocations = targets == null ? 0 : targets.length; - this.replicas = new ArrayList(numLocations); - for(int i = 0; i < numLocations; i++) - replicas.add( - new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW)); + this.replicas = new ArrayList<>(numLocations); + for(int i = 0; i < numLocations; i++) { + replicas.add(new ReplicaUnderConstruction(this, targets[i], + ReplicaState.RBW)); + } } /** @@ -204,8 +111,9 @@ public void setExpectedLocations(DatanodeStorageInfo[] targets) { public DatanodeStorageInfo[] getExpectedStorageLocations() { int numLocations = replicas == null ? 0 : replicas.size(); DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; - for(int i = 0; i < numLocations; i++) + for (int i = 0; i < numLocations; i++) { storages[i] = replicas.get(i).getExpectedStorageLocation(); + } return storages; } @@ -293,17 +201,17 @@ public void initializeBlockRecovery(long recoveryId) { + " No blocks found, lease removed."); } boolean allLiveReplicasTriedAsPrimary = true; - for (int i = 0; i < replicas.size(); i++) { + for (ReplicaUnderConstruction replica : replicas) { // Check if all replicas have been tried or not. - if (replicas.get(i).isAlive()) { - allLiveReplicasTriedAsPrimary = - (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary()); + if (replica.isAlive()) { + allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary && + replica.getChosenAsPrimary()); } } if (allLiveReplicasTriedAsPrimary) { // Just set all the replicas to be chosen whether they are alive or not. - for (int i = 0; i < replicas.size(); i++) { - replicas.get(i).setChosenAsPrimary(false); + for (ReplicaUnderConstruction replica : replicas) { + replica.setChosenAsPrimary(false); } } long mostRecentLastUpdate = 0; @@ -324,7 +232,8 @@ public void initializeBlockRecovery(long recoveryId) { } } if (primary != null) { - primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this); + primary.getExpectedStorageLocation().getDatanodeDescriptor() + .addBlockToBeRecovered(this); primary.setChosenAsPrimary(true); NameNode.blockStateChangeLog.info( "BLOCK* {} recovery started, primary={}", this, primary); @@ -357,18 +266,6 @@ void addReplicaIfNotPresent(DatanodeStorageInfo storage, replicas.add(new ReplicaUnderConstruction(block, storage, rState)); } - @Override // BlockInfo - // BlockInfoUnderConstruction participates in maps the same way as BlockInfo - public int hashCode() { - return super.hashCode(); - } - - @Override // BlockInfo - public boolean equals(Object obj) { - // Sufficient to rely on super's implementation - return (this == obj) || super.equals(obj); - } - @Override public String toString() { final StringBuilder b = new StringBuilder(100); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java new file mode 100644 index 0000000000000..5fff41e59f7bd --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -0,0 +1,179 @@ +/** + * 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 org.apache.hadoop.hdfs.protocol.Block; + +/** + * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. + * + * We still use triplets to store DatanodeStorageInfo for each block in the + * block group, as well as the previous/next block in the corresponding + * DatanodeStorageInfo. For a (m+k) block group, the first (m+k) triplet units + * are sorted and strictly mapped to the corresponding block. + * + * Normally each block belonging to group is stored in only one DataNode. + * However, it is possible that some block is over-replicated. Thus the triplet + * array's size can be larger than (m+k). Thus currently we use an extra byte + * array to record the block index for each triplet. + */ +public class BlockInfoStriped extends BlockInfo { + private final short dataBlockNum; + private final short parityBlockNum; + /** + * Always the same size with triplets. Record the block index for each triplet + * TODO: actually this is only necessary for over-replicated block. Thus can + * be further optimized to save memory usage. + */ + private byte[] indices; + + public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) { + super(blk, (short) (dataBlockNum + parityBlockNum)); + indices = new byte[dataBlockNum + parityBlockNum]; + initIndices(); + this.dataBlockNum = dataBlockNum; + this.parityBlockNum = parityBlockNum; + } + + BlockInfoStriped(BlockInfoStriped b) { + this(b, b.dataBlockNum, b.parityBlockNum); + this.setBlockCollection(b.getBlockCollection()); + } + + private short getTotalBlockNum() { + return (short) (dataBlockNum + parityBlockNum); + } + + private void initIndices() { + for (int i = 0; i < indices.length; i++) { + indices[i] = -1; + } + } + + private int findSlot() { + int i = getTotalBlockNum(); + for (; i < getCapacity(); i++) { + if (getStorageInfo(i) == null) { + return i; + } + } + // need to expand the triplet size + ensureCapacity(i + 1, true); + return i; + } + + @Override + boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) { + int blockIndex = BlockIdManager.getBlockIndex(reportedBlock); + int index = blockIndex; + DatanodeStorageInfo old = getStorageInfo(index); + if (old != null && !old.equals(storage)) { // over replicated + // check if the storage has been stored + int i = findStorageInfo(storage); + if (i == -1) { + index = findSlot(); + } else { + return true; + } + } + addStorage(storage, index, blockIndex); + return true; + } + + private void addStorage(DatanodeStorageInfo storage, int index, + int blockIndex) { + setStorageInfo(index, storage); + setNext(index, null); + setPrevious(index, null); + indices[index] = (byte) blockIndex; + } + + private int findStorageInfoFromEnd(DatanodeStorageInfo storage) { + final int len = getCapacity(); + for(int idx = len - 1; idx >= 0; idx--) { + DatanodeStorageInfo cur = getStorageInfo(idx); + if (storage.equals(cur)) { + return idx; + } + } + return -1; + } + + @Override + boolean removeStorage(DatanodeStorageInfo storage) { + int dnIndex = findStorageInfoFromEnd(storage); + if (dnIndex < 0) { // the node is not found + return false; + } + assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : + "Block is still in the list and must be removed first."; + // set the triplet to null + setStorageInfo(dnIndex, null); + setNext(dnIndex, null); + setPrevious(dnIndex, null); + indices[dnIndex] = -1; + return true; + } + + private void ensureCapacity(int totalSize, boolean keepOld) { + if (getCapacity() < totalSize) { + Object[] old = triplets; + byte[] oldIndices = indices; + triplets = new Object[totalSize * 3]; + indices = new byte[totalSize]; + initIndices(); + + if (keepOld) { + System.arraycopy(old, 0, triplets, 0, old.length); + System.arraycopy(oldIndices, 0, indices, 0, oldIndices.length); + } + } + } + + @Override + void replaceBlock(BlockInfo newBlock) { + assert newBlock instanceof BlockInfoStriped; + BlockInfoStriped newBlockGroup = (BlockInfoStriped) newBlock; + final int size = getCapacity(); + newBlockGroup.ensureCapacity(size, false); + for (int i = 0; i < size; i++) { + final DatanodeStorageInfo storage = this.getStorageInfo(i); + if (storage != null) { + final int blockIndex = indices[i]; + final boolean removed = storage.removeBlock(this); + assert removed : "currentBlock not found."; + + newBlockGroup.addStorage(storage, i, blockIndex); + storage.insertToList(newBlockGroup); + } + } + } + + @Override + public int numNodes() { + assert this.triplets != null : "BlockInfo is not initialized"; + assert triplets.length % 3 == 0 : "Malformed BlockInfo"; + int num = 0; + for (int idx = getCapacity()-1; idx >= 0; idx--) { + if (getStorageInfo(idx) != null) { + num++; + } + } + return num; + } +} 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 dda19434b84ec..6c0fb3031d4c3 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 @@ -599,8 +599,8 @@ public boolean checkMinReplication(Block block) { * of replicas reported from data-nodes. */ private static boolean commitBlock( - final BlockInfoContiguousUnderConstruction block, final Block commitBlock) - throws IOException { + final BlockInfoContiguousUnderConstruction block, + final Block commitBlock) throws IOException { if (block.getBlockUCState() == BlockUCState.COMMITTED) return false; assert block.getNumBytes() <= commitBlock.getNumBytes() : @@ -631,7 +631,7 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, return false; // already completed (e.g. by syncBlock) final boolean b = commitBlock( - (BlockInfoContiguousUnderConstruction) lastBlock, commitBlock); + (BlockInfoContiguousUnderConstruction)lastBlock, commitBlock); if(countNodes(lastBlock).liveReplicas() >= minReplication) completeBlock(bc, bc.numBlocks()-1, false); return b; @@ -644,15 +644,16 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, * @throws IOException if the block does not have at least a minimal number * of replicas reported from data-nodes. */ - private BlockInfoContiguous completeBlock(final BlockCollection bc, + private BlockInfo completeBlock(final BlockCollection bc, final int blkIndex, boolean force) throws IOException { if(blkIndex < 0) return null; BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex]; - if(curBlock.isComplete()) + if (curBlock.isComplete()) return curBlock; + // TODO: support BlockInfoStripedUC BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction) curBlock; + (BlockInfoContiguousUnderConstruction)curBlock; int numNodes = ucBlock.numNodes(); if (!force && numNodes < minReplication) throw new IOException("Cannot complete block: " + @@ -678,13 +679,15 @@ private BlockInfoContiguous completeBlock(final BlockCollection bc, return blocksMap.replaceBlock(completeBlock); } - private BlockInfoContiguous completeBlock(final BlockCollection bc, - final BlockInfoContiguous block, boolean force) throws IOException { + // TODO: support BlockInfoStrippedUC + private BlockInfo completeBlock(final BlockCollection bc, + final BlockInfo block, boolean force) throws IOException { BlockInfoContiguous[] fileBlocks = bc.getBlocks(); - for(int idx = 0; idx < fileBlocks.length; idx++) - if(fileBlocks[idx] == block) { + for (int idx = 0; idx < fileBlocks.length; idx++) { + if (fileBlocks[idx] == block) { return completeBlock(bc, idx, force); } + } return block; } @@ -693,7 +696,7 @@ private BlockInfoContiguous completeBlock(final BlockCollection bc, * regardless of whether enough replicas are present. This is necessary * when tailing edit logs as a Standby. */ - public BlockInfoContiguous forceCompleteBlock(final BlockCollection bc, + public BlockInfo forceCompleteBlock(final BlockCollection bc, final BlockInfoContiguousUnderConstruction block) throws IOException { block.commitBlock(block); return completeBlock(bc, block, true); @@ -725,8 +728,8 @@ public LocatedBlock convertLastBlockToUnderConstruction( DatanodeStorageInfo[] targets = getStorages(oldBlock); - BlockInfoContiguousUnderConstruction ucBlock = - bc.setLastBlock(oldBlock, targets); + BlockInfoContiguousUnderConstruction ucBlock = bc.setLastBlock(oldBlock, + targets); blocksMap.replaceBlock(ucBlock); // Remove block from replication queue. @@ -1027,7 +1030,7 @@ private BlocksWithLocations getBlocksWithLocations(final DatanodeID datanode, if(numBlocks == 0) { return new BlocksWithLocations(new BlockWithLocations[0]); } - Iterator iter = node.getBlockIterator(); + Iterator iter = node.getBlockIterator(); // starting from a random block int startBlock = ThreadLocalRandom.current().nextInt(numBlocks); // skip blocks @@ -1036,7 +1039,7 @@ private BlocksWithLocations getBlocksWithLocations(final DatanodeID datanode, } List results = new ArrayList(); long totalSize = 0; - BlockInfoContiguous curBlock; + BlockInfo curBlock; while(totalSize processReport( // Modify the (block-->datanode) map, according to the difference // between the old and new block report. // - Collection toAdd = new LinkedList(); + Collection toAdd = new LinkedList<>(); Collection toRemove = new TreeSet(); Collection toInvalidate = new LinkedList(); Collection toCorrupt = new LinkedList(); @@ -2000,8 +2018,9 @@ private Collection processReport( removeStoredBlock(b, node); } int numBlocksLogged = 0; - for (BlockInfoContiguous b : toAdd) { - addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog); + for (BlockInfoToAdd b : toAdd) { + addStoredBlock(b.stored, b.reported, storageInfo, null, + numBlocksLogged < maxNumBlocksToLog); numBlocksLogged++; } if (numBlocksLogged > maxNumBlocksToLog) { @@ -2091,7 +2110,7 @@ private void processFirstBlockReport( continue; } - BlockInfoContiguous storedBlock = getStoredBlock(iblk); + BlockInfo storedBlock = getStoredBlock(iblk); // If block does not belong to any file, we are done. if (storedBlock == null) continue; @@ -2114,7 +2133,7 @@ private void processFirstBlockReport( // If block is under construction, add this replica to its list if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) { - ((BlockInfoContiguousUnderConstruction)storedBlock) + ((BlockInfoContiguousUnderConstruction) storedBlock) .addReplicaIfNotPresent(storageInfo, iblk, reportedState); // OpenFileBlocks only inside snapshots also will be added to safemode // threshold. So we need to update such blocks to safemode @@ -2129,14 +2148,14 @@ private void processFirstBlockReport( } //add replica if appropriate if (reportedState == ReplicaState.FINALIZED) { - addStoredBlockImmediate(storedBlock, storageInfo); + addStoredBlockImmediate(storedBlock, iblk, storageInfo); } } } private void reportDiff(DatanodeStorageInfo storageInfo, BlockListAsLongs newReport, - Collection toAdd, // add to DatanodeDescriptor + Collection toAdd, // add to DatanodeDescriptor Collection toRemove, // remove from DatanodeDescriptor Collection toInvalidate, // should be removed from DN Collection toCorrupt, // add to corrupt replicas list @@ -2144,8 +2163,10 @@ private void reportDiff(DatanodeStorageInfo storageInfo, // place a delimiter in the list which separates blocks // that have been reported from those that have not - BlockInfoContiguous delimiter = new BlockInfoContiguous(new Block(), (short) 1); - AddBlockResult result = storageInfo.addBlock(delimiter); + Block delimiterBlock = new Block(); + BlockInfoContiguous delimiter = new BlockInfoContiguous(delimiterBlock, + (short) 1); + AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock); assert result == AddBlockResult.ADDED : "Delimiting block cannot be present in the node"; int headIndex = 0; //currently the delimiter is in the head of the list @@ -2157,7 +2178,7 @@ private void reportDiff(DatanodeStorageInfo storageInfo, // scan the report and process newly reported blocks for (BlockReportReplica iblk : newReport) { ReplicaState iState = iblk.getState(); - BlockInfoContiguous storedBlock = processReportedBlock(storageInfo, + BlockInfo storedBlock = processReportedBlock(storageInfo, iblk, iState, toAdd, toInvalidate, toCorrupt, toUC); // move block to the head of the list @@ -2169,8 +2190,7 @@ private void reportDiff(DatanodeStorageInfo storageInfo, // collect blocks that have not been reported // all of them are next to the delimiter - Iterator it = - storageInfo.new BlockIterator(delimiter.getNext(0)); + Iterator it = storageInfo.new BlockIterator(delimiter.getNext(0)); while(it.hasNext()) toRemove.add(it.next()); storageInfo.removeBlock(delimiter); @@ -2207,10 +2227,10 @@ private void reportDiff(DatanodeStorageInfo storageInfo, * @return the up-to-date stored block, if it should be kept. * Otherwise, null. */ - private BlockInfoContiguous processReportedBlock( + private BlockInfo processReportedBlock( final DatanodeStorageInfo storageInfo, final Block block, final ReplicaState reportedState, - final Collection toAdd, + final Collection toAdd, final Collection toInvalidate, final Collection toCorrupt, final Collection toUC) { @@ -2231,7 +2251,7 @@ private BlockInfoContiguous processReportedBlock( } // find block by blockId - BlockInfoContiguous storedBlock = getStoredBlock(block); + BlockInfo storedBlock = getStoredBlock(block); if(storedBlock == null) { // If blocksMap does not contain reported block id, // the replica should be removed from the data-node. @@ -2285,7 +2305,7 @@ private BlockInfoContiguous processReportedBlock( if (reportedState == ReplicaState.FINALIZED && (storedBlock.findStorageInfo(storageInfo) == -1 || corruptReplicas.isReplicaCorrupt(storedBlock, dn))) { - toAdd.add(storedBlock); + toAdd.add(new BlockInfoToAdd(storedBlock, block)); } return storedBlock; } @@ -2370,7 +2390,7 @@ public void processAllPendingDNMessages() throws IOException { */ private BlockToMarkCorrupt checkReplicaCorrupt( Block reported, ReplicaState reportedState, - BlockInfoContiguous storedBlock, BlockUCState ucState, + BlockInfo storedBlock, BlockUCState ucState, DatanodeDescriptor dn) { switch(reportedState) { case FINALIZED: @@ -2379,12 +2399,12 @@ private BlockToMarkCorrupt checkReplicaCorrupt( case COMMITTED: if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(storedBlock, reportedGS, + return new BlockToMarkCorrupt(reported, storedBlock, reportedGS, "block is " + ucState + " and reported genstamp " + reportedGS + " does not match genstamp in block map " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); } else if (storedBlock.getNumBytes() != reported.getNumBytes()) { - return new BlockToMarkCorrupt(storedBlock, + return new BlockToMarkCorrupt(reported, storedBlock, "block is " + ucState + " and reported length " + reported.getNumBytes() + " does not match " + "length in block map " + storedBlock.getNumBytes(), @@ -2395,8 +2415,8 @@ private BlockToMarkCorrupt checkReplicaCorrupt( case UNDER_CONSTRUCTION: if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(storedBlock, reportedGS, "block is " - + ucState + " and reported state " + reportedState + return new BlockToMarkCorrupt(reported, storedBlock, reportedGS, + "block is " + ucState + " and reported state " + reportedState + ", But reported genstamp " + reportedGS + " does not match genstamp in block map " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); @@ -2411,7 +2431,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( return null; // not corrupt } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(storedBlock, reportedGS, + return new BlockToMarkCorrupt(reported, storedBlock, reportedGS, "reported " + reportedState + " replica with genstamp " + reportedGS + " does not match COMPLETE block's genstamp in block map " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); @@ -2426,7 +2446,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( "complete with the same genstamp"); return null; } else { - return new BlockToMarkCorrupt(storedBlock, + return new BlockToMarkCorrupt(reported, storedBlock, "reported replica has invalid state " + reportedState, Reason.INVALID_STATE); } @@ -2439,11 +2459,12 @@ private BlockToMarkCorrupt checkReplicaCorrupt( " on " + dn + " size " + storedBlock.getNumBytes(); // log here at WARN level since this is really a broken HDFS invariant LOG.warn(msg); - return new BlockToMarkCorrupt(storedBlock, msg, Reason.INVALID_STATE); + return new BlockToMarkCorrupt(reported, storedBlock, msg, + Reason.INVALID_STATE); } } - private boolean isBlockUnderConstruction(BlockInfoContiguous storedBlock, + private boolean isBlockUnderConstruction(BlockInfo storedBlock, BlockUCState ucState, ReplicaState reportedState) { switch(reportedState) { case FINALIZED: @@ -2472,7 +2493,7 @@ void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, if (ucBlock.reportedState == ReplicaState.FINALIZED && (block.findStorageInfo(storageInfo) < 0)) { - addStoredBlock(block, storageInfo, null, true); + addStoredBlock(block, ucBlock.reportedBlock, storageInfo, null, true); } } @@ -2487,18 +2508,18 @@ void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, * * @throws IOException */ - private void addStoredBlockImmediate(BlockInfoContiguous storedBlock, + private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported, DatanodeStorageInfo storageInfo) throws IOException { assert (storedBlock != null && namesystem.hasWriteLock()); if (!namesystem.isInStartupSafeMode() || namesystem.isPopulatingReplQueues()) { - addStoredBlock(storedBlock, storageInfo, null, false); + addStoredBlock(storedBlock, reported, storageInfo, null, false); return; } // just add it - AddBlockResult result = storageInfo.addBlock(storedBlock); + AddBlockResult result = storageInfo.addBlock(storedBlock, reported); // Now check for completion of blocks and safe block count int numCurrentReplica = countLiveNodes(storedBlock); @@ -2519,13 +2540,14 @@ private void addStoredBlockImmediate(BlockInfoContiguous storedBlock, * needed replications if this takes care of the problem. * @return the block that is stored in blockMap. */ - private Block addStoredBlock(final BlockInfoContiguous block, + private Block addStoredBlock(final BlockInfo block, + final Block reportedBlock, DatanodeStorageInfo storageInfo, DatanodeDescriptor delNodeHint, boolean logEveryBlock) throws IOException { assert block != null && namesystem.hasWriteLock(); - BlockInfoContiguous storedBlock; + BlockInfo storedBlock; DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); if (block instanceof BlockInfoContiguousUnderConstruction) { //refresh our copy in case the block got completed in another thread @@ -2546,7 +2568,7 @@ private Block addStoredBlock(final BlockInfoContiguous block, assert bc != null : "Block must belong to a file"; // add block to the datanode - AddBlockResult result = storageInfo.addBlock(storedBlock); + AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock); int curReplicaDelta; if (result == AddBlockResult.ADDED) { @@ -2618,13 +2640,13 @@ private Block addStoredBlock(final BlockInfoContiguous block, storedBlock + "blockMap has " + numCorruptNodes + " but corrupt replicas map has " + corruptReplicasCount); } - if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) - invalidateCorruptReplicas(storedBlock); + if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) { + invalidateCorruptReplicas(storedBlock, reportedBlock); + } return storedBlock; } - private void logAddStoredBlock(BlockInfoContiguous storedBlock, - DatanodeDescriptor node) { + private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) { if (!blockLog.isInfoEnabled()) { return; } @@ -2651,7 +2673,7 @@ private void logAddStoredBlock(BlockInfoContiguous storedBlock, * * @param blk Block whose corrupt replicas need to be invalidated */ - private void invalidateCorruptReplicas(BlockInfoContiguous blk) { + private void invalidateCorruptReplicas(BlockInfo blk, Block reported) { Collection nodes = corruptReplicas.getNodes(blk); boolean removedFromBlocksMap = true; if (nodes == null) @@ -2661,7 +2683,7 @@ private void invalidateCorruptReplicas(BlockInfoContiguous blk) { DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]); for (DatanodeDescriptor node : nodesCopy) { try { - if (!invalidateBlock(new BlockToMarkCorrupt(blk, null, + if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null, Reason.ANY), node)) { removedFromBlocksMap = false; } @@ -2730,7 +2752,7 @@ private void processMisReplicatesAsync() throws InterruptedException { long nrInvalid = 0, nrOverReplicated = 0; long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0; long startTimeMisReplicatedScan = Time.monotonicNow(); - Iterator blocksItr = blocksMap.getBlocks().iterator(); + Iterator blocksItr = blocksMap.getBlocks().iterator(); long totalBlocks = blocksMap.size(); replicationQueuesInitProgress = 0; long totalProcessed = 0; @@ -2742,7 +2764,7 @@ private void processMisReplicatesAsync() throws InterruptedException { namesystem.writeLockInterruptibly(); try { while (processed < numBlocksPerIteration && blocksItr.hasNext()) { - BlockInfoContiguous block = blocksItr.next(); + BlockInfo block = blocksItr.next(); MisReplicationResult res = processMisReplicatedBlock(block); if (LOG.isTraceEnabled()) { LOG.trace("block " + block + ": " + res); @@ -2817,7 +2839,7 @@ public double getReplicationQueuesInitProgress() { * appropriate queues if necessary, and returns a result code indicating * what happened with it. */ - private MisReplicationResult processMisReplicatedBlock(BlockInfoContiguous block) { + private MisReplicationResult processMisReplicatedBlock(BlockInfo block) { if (block.isDeleted()) { // block does not belong to any file addToInvalidates(block); @@ -3157,14 +3179,14 @@ private void processAndHandleReportedBlock( ReplicaState reportedState, DatanodeDescriptor delHintNode) throws IOException { // blockReceived reports a finalized block - Collection toAdd = new LinkedList(); + Collection toAdd = new LinkedList<>(); Collection toInvalidate = new LinkedList(); Collection toCorrupt = new LinkedList(); Collection toUC = new LinkedList(); final DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); - processReportedBlock(storageInfo, block, reportedState, - toAdd, toInvalidate, toCorrupt, toUC); + processReportedBlock(storageInfo, 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 assert toUC.size() + toAdd.size() + toInvalidate.size() + toCorrupt.size() <= 1 @@ -3174,8 +3196,9 @@ private void processAndHandleReportedBlock( addStoredBlockUnderConstruction(b, storageInfo); } long numBlocksLogged = 0; - for (BlockInfoContiguous b : toAdd) { - addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog); + for (BlockInfoToAdd b : toAdd) { + addStoredBlock(b.stored, b.reported, storageInfo, delHintNode, + numBlocksLogged < maxNumBlocksToLog); numBlocksLogged++; } if (numBlocksLogged > maxNumBlocksToLog) { @@ -3301,7 +3324,7 @@ public NumberReplicas countNodes(Block b) { * @param b - the block being tested * @return count of live nodes for this block */ - int countLiveNodes(BlockInfoContiguous b) { + int countLiveNodes(BlockInfo b) { if (!namesystem.isInStartupSafeMode()) { return countNodes(b).liveReplicas(); } @@ -3380,7 +3403,7 @@ public int getActiveBlockCount() { return blocksMap.size(); } - public DatanodeStorageInfo[] getStorages(BlockInfoContiguous block) { + public DatanodeStorageInfo[] getStorages(BlockInfo block) { final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()]; int i = 0; for(DatanodeStorageInfo s : blocksMap.getStorages(block)) { @@ -3409,8 +3432,8 @@ public void removeBlock(Block block) { } } - public BlockInfoContiguous getStoredBlock(Block block) { - BlockInfoContiguous info = null; + public BlockInfo getStoredBlock(Block block) { + BlockInfo info = null; if (BlockIdManager.isStripedBlockID(block.getBlockId())) { info = blocksMap.getStoredBlock( new Block(BlockIdManager.convertToGroupID(block.getBlockId()))); @@ -3588,7 +3611,8 @@ public long getMissingReplOneBlocksCount() { public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block, BlockCollection bc) { - return blocksMap.addBlockCollection(block, bc); + // TODO + return (BlockInfoContiguous) blocksMap.addBlockCollection(block, bc); } public BlockCollection getBlockCollection(Block b) { @@ -3826,7 +3850,7 @@ private void chooseTargets(BlockPlacementPolicy blockplacement, /** * A simple result enum for the result of - * {@link BlockManager#processMisReplicatedBlock(BlockInfoContiguous)}. + * {@link BlockManager#processMisReplicatedBlock}. */ enum MisReplicationResult { /** The block should be invalidated since it belongs to a deleted file. */ 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 5e7d34f0a0e76..59ff030b89d90 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 @@ -20,12 +20,10 @@ import java.util.Iterator; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.util.GSet; import org.apache.hadoop.util.LightWeightGSet; -import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; @@ -36,10 +34,10 @@ */ class BlocksMap { private static class StorageIterator implements Iterator { - private final BlockInfoContiguous blockInfo; + private final BlockInfo blockInfo; private int nextIdx = 0; - StorageIterator(BlockInfoContiguous blkInfo) { + StorageIterator(BlockInfo blkInfo) { this.blockInfo = blkInfo; } @@ -63,14 +61,14 @@ public void remove() { /** Constant {@link LightWeightGSet} capacity. */ private final int capacity; - private GSet blocks; + private GSet blocks; BlocksMap(int capacity) { // Use 2% of total memory to size the GSet capacity this.capacity = capacity; - this.blocks = new LightWeightGSet(capacity) { + this.blocks = new LightWeightGSet(capacity) { @Override - public Iterator iterator() { + public Iterator iterator() { SetIterator iterator = new SetIterator(); /* * Not tracking any modifications to set. As this set will be used @@ -97,15 +95,15 @@ void clear() { } BlockCollection getBlockCollection(Block b) { - BlockInfoContiguous info = blocks.get(b); + BlockInfo info = blocks.get(b); return (info != null) ? info.getBlockCollection() : null; } /** * Add block b belonging to the specified block collection to the map. */ - BlockInfoContiguous addBlockCollection(BlockInfoContiguous b, BlockCollection bc) { - BlockInfoContiguous info = blocks.get(b); + BlockInfo addBlockCollection(BlockInfo b, BlockCollection bc) { + BlockInfo info = blocks.get(b); if (info != b) { info = b; blocks.put(info); @@ -120,11 +118,12 @@ BlockInfoContiguous addBlockCollection(BlockInfoContiguous b, BlockCollection bc * and remove all data-node locations associated with the block. */ void removeBlock(Block block) { - BlockInfoContiguous blockInfo = blocks.remove(block); + BlockInfo blockInfo = blocks.remove(block); if (blockInfo == null) return; blockInfo.setBlockCollection(null); + // TODO: fix this logic for block group for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) { DatanodeDescriptor dn = blockInfo.getDatanode(idx); dn.removeBlock(blockInfo); // remove from the list and wipe the location @@ -132,7 +131,7 @@ void removeBlock(Block block) { } /** Returns the block object it it exists in the map. */ - BlockInfoContiguous getStoredBlock(Block b) { + BlockInfo getStoredBlock(Block b) { return blocks.get(b); } @@ -164,7 +163,7 @@ public boolean apply(DatanodeStorageInfo storage) { * For a block that has already been retrieved from the BlocksMap * returns {@link Iterable} of the storages the block belongs to. */ - Iterable getStorages(final BlockInfoContiguous storedBlock) { + Iterable getStorages(final BlockInfo storedBlock) { return new Iterable() { @Override public Iterator iterator() { @@ -175,7 +174,7 @@ public Iterator iterator() { /** counts number of containing nodes. Better than using iterator. */ int numNodes(Block b) { - BlockInfoContiguous info = blocks.get(b); + BlockInfo info = blocks.get(b); return info == null ? 0 : info.numNodes(); } @@ -185,7 +184,7 @@ int numNodes(Block b) { * only if it does not belong to any file and data-nodes. */ boolean removeNode(Block b, DatanodeDescriptor node) { - BlockInfoContiguous info = blocks.get(b); + BlockInfo info = blocks.get(b); if (info == null) return false; @@ -203,7 +202,7 @@ int size() { return blocks.size(); } - Iterable getBlocks() { + Iterable getBlocks() { return blocks; } @@ -218,20 +217,11 @@ int getCapacity() { * @param newBlock - block for replacement * @return new block */ - BlockInfoContiguous replaceBlock(BlockInfoContiguous newBlock) { - BlockInfoContiguous currentBlock = blocks.get(newBlock); + BlockInfo replaceBlock(BlockInfo newBlock) { + BlockInfo currentBlock = blocks.get(newBlock); assert currentBlock != null : "the block if not in blocksMap"; // replace block in data-node lists - for (int i = currentBlock.numNodes() - 1; i >= 0; i--) { - final DatanodeDescriptor dn = currentBlock.getDatanode(i); - final DatanodeStorageInfo storage = currentBlock.findStorageInfo(dn); - final boolean removed = storage.removeBlock(currentBlock); - Preconditions.checkState(removed, "currentBlock not found."); - - final AddBlockResult result = storage.addBlock(newBlock); - Preconditions.checkState(result == AddBlockResult.ADDED, - "newBlock already exists."); - } + currentBlock.replaceBlock(newBlock); // replace block in the map itself blocks.put(newBlock); return newBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java index bf5ece9bc1618..79d77137f2904 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java @@ -513,8 +513,7 @@ private void rescanCachedBlockMap() { iter.remove(); } } - BlockInfoContiguous blockInfo = blockManager. - getStoredBlock(new Block(cblock.getBlockId())); + BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId())); String reason = findReasonForNotCaching(cblock, blockInfo); int neededCached = 0; if (reason != null) { @@ -628,8 +627,7 @@ private void addNewPendingCached(final int neededCached, List pendingCached) { // To figure out which replicas can be cached, we consult the // blocksMap. We don't want to try to cache a corrupt replica, though. - BlockInfoContiguous blockInfo = blockManager. - getStoredBlock(new Block(cachedBlock.getBlockId())); + BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId())); if (blockInfo == null) { LOG.debug("Block {}: can't add new cached replicas," + " because there is no record of this block " + @@ -668,7 +666,7 @@ private void addNewPendingCached(final int neededCached, while (it.hasNext()) { CachedBlock cBlock = it.next(); BlockInfoContiguous info = - blockManager.getStoredBlock(new Block(cBlock.getBlockId())); + namesystem.getStoredBlock(new Block(cBlock.getBlockId())); if (info != null) { pendingBytes -= info.getNumBytes(); } @@ -678,7 +676,7 @@ private void addNewPendingCached(final int neededCached, while (it.hasNext()) { CachedBlock cBlock = it.next(); BlockInfoContiguous info = - blockManager.getStoredBlock(new Block(cBlock.getBlockId())); + namesystem.getStoredBlock(new Block(cBlock.getBlockId())); if (info != null) { pendingBytes += info.getNumBytes(); } 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 4731ad44c3116..415646a5ec9e1 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 @@ -335,7 +335,7 @@ List removeZombieStorages() { * Remove block from the list of blocks belonging to the data-node. Remove * data-node from the block. */ - boolean removeBlock(BlockInfoContiguous b) { + boolean removeBlock(BlockInfo b) { final DatanodeStorageInfo s = b.findStorageInfo(this); // if block exists on this datanode if (s != null) { @@ -348,12 +348,9 @@ boolean removeBlock(BlockInfoContiguous b) { * Remove block from the list of blocks belonging to the data-node. Remove * data-node from the block. */ - boolean removeBlock(String storageID, BlockInfoContiguous b) { + boolean removeBlock(String storageID, BlockInfo b) { DatanodeStorageInfo s = getStorageInfo(storageID); - if (s != null) { - return s.removeBlock(b); - } - return false; + return s != null && s.removeBlock(b); } public void resetBlocks() { @@ -537,12 +534,12 @@ private void updateFailedStorage( } } - private static class BlockIterator implements Iterator { + private static class BlockIterator implements Iterator { private int index = 0; - private final List> iterators; + private final List> iterators; private BlockIterator(final DatanodeStorageInfo... storages) { - List> iterators = new ArrayList>(); + List> iterators = new ArrayList<>(); for (DatanodeStorageInfo e : storages) { iterators.add(e.getBlockIterator()); } @@ -556,7 +553,7 @@ public boolean hasNext() { } @Override - public BlockInfoContiguous next() { + public BlockInfo next() { update(); return iterators.get(index).next(); } @@ -573,10 +570,11 @@ private void update() { } } - Iterator getBlockIterator() { + Iterator getBlockIterator() { return new BlockIterator(getStorageInfos()); } - Iterator getBlockIterator(final String storageID) { + + Iterator getBlockIterator(final String storageID) { return new BlockIterator(getStorageInfo(storageID)); } 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 index c6c9001d1d920..2c8b3eaba708e 100644 --- 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 @@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -83,10 +84,10 @@ public void updateFromStorage(DatanodeStorage storage) { /** * Iterates over the list of blocks belonging to the data-node. */ - class BlockIterator implements Iterator { - private BlockInfoContiguous current; + class BlockIterator implements Iterator { + private BlockInfo current; - BlockIterator(BlockInfoContiguous head) { + BlockIterator(BlockInfo head) { this.current = head; } @@ -94,8 +95,8 @@ public boolean hasNext() { return current != null; } - public BlockInfoContiguous next() { - BlockInfoContiguous res = current; + public BlockInfo next() { + BlockInfo res = current; current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this)); return res; } @@ -115,7 +116,7 @@ public void remove() { private volatile long remaining; private long blockPoolUsed; - private volatile BlockInfoContiguous blockList = null; + private volatile BlockInfo blockList = null; private int numBlocks = 0; // The ID of the last full block report which updated this storage. @@ -229,7 +230,7 @@ long getBlockPoolUsed() { return blockPoolUsed; } - public AddBlockResult addBlock(BlockInfoContiguous b) { + public AddBlockResult addBlock(BlockInfo b, Block reportedBlock) { // First check whether the block belongs to a different storage // on the same DN. AddBlockResult result = AddBlockResult.ADDED; @@ -248,13 +249,21 @@ public AddBlockResult addBlock(BlockInfoContiguous b) { } // add to the head of the data-node list - b.addStorage(this); + b.addStorage(this, reportedBlock); + insertToList(b); + return result; + } + + AddBlockResult addBlock(BlockInfoContiguous b) { + return addBlock(b, b); + } + + public void insertToList(BlockInfo b) { blockList = b.listInsert(blockList, this); numBlocks++; - return result; } - public boolean removeBlock(BlockInfoContiguous b) { + public boolean removeBlock(BlockInfo b) { blockList = b.listRemove(blockList, this); if (b.removeStorage(this)) { numBlocks--; @@ -268,16 +277,15 @@ int numBlocks() { return numBlocks; } - Iterator getBlockIterator() { + 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(BlockInfoContiguous b, int curIndex, int headIndex) { + int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) { blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex); return curIndex; } @@ -287,7 +295,7 @@ int moveBlockToHead(BlockInfoContiguous b, int curIndex, int headIndex) { * @return the head of the blockList */ @VisibleForTesting - BlockInfoContiguous getBlockListHeadForTesting(){ + BlockInfo getBlockListHeadForTesting(){ return blockList; } @@ -374,6 +382,6 @@ static DatanodeStorageInfo getDatanodeStorageInfo( } static enum AddBlockResult { - ADDED, REPLACED, ALREADY_EXIST; + ADDED, REPLACED, ALREADY_EXIST } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java new file mode 100644 index 0000000000000..f4600cb74fc0b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java @@ -0,0 +1,119 @@ +/** + * 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 org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; + +/** + * ReplicaUnderConstruction contains information about replicas (or blocks + * belonging to a block group) while they are under construction. + * + * The GS, the length and the state of the replica is as reported by the + * datanode. + * + * It is not guaranteed, but expected, that datanodes actually have + * corresponding replicas. + */ +class ReplicaUnderConstruction extends Block { + private final DatanodeStorageInfo expectedLocation; + private HdfsServerConstants.ReplicaState state; + private boolean chosenAsPrimary; + + ReplicaUnderConstruction(Block block, + DatanodeStorageInfo target, + HdfsServerConstants.ReplicaState state) { + super(block); + this.expectedLocation = target; + this.state = state; + this.chosenAsPrimary = false; + } + + /** + * Expected block replica location as assigned when the block was allocated. + * This defines the pipeline order. + * It is not guaranteed, but expected, that the data-node actually has + * the replica. + */ + DatanodeStorageInfo getExpectedStorageLocation() { + return expectedLocation; + } + + /** + * Get replica state as reported by the data-node. + */ + HdfsServerConstants.ReplicaState getState() { + return state; + } + + /** + * Whether the replica was chosen for recovery. + */ + boolean getChosenAsPrimary() { + return chosenAsPrimary; + } + + /** + * Set replica state. + */ + void setState(HdfsServerConstants.ReplicaState s) { + state = s; + } + + /** + * Set whether this replica was chosen for recovery. + */ + void setChosenAsPrimary(boolean chosenAsPrimary) { + this.chosenAsPrimary = chosenAsPrimary; + } + + /** + * Is data-node the replica belongs to alive. + */ + boolean isAlive() { + return expectedLocation.getDatanodeDescriptor().isAlive; + } + + @Override // Block + public int hashCode() { + return super.hashCode(); + } + + @Override // Block + public boolean equals(Object obj) { + // Sufficient to rely on super's implementation + return (this == obj) || super.equals(obj); + } + + @Override + public String toString() { + final StringBuilder b = new StringBuilder(50); + appendStringTo(b); + return b.toString(); + } + + @Override + public void appendStringTo(StringBuilder sb) { + sb.append("ReplicaUC[") + .append(expectedLocation) + .append("|") + .append(state) + .append("]"); + } +} + 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 0d229e1af451d..d0313dbec30c3 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 @@ -3072,7 +3072,7 @@ boolean checkFileProgress(String src, INodeFile v, boolean checkall) { src, new BlockInfoContiguous[] { b }); } } - + /** * Change the indicated filename. * @deprecated Use {@link #renameTo(String, String, boolean, @@ -3540,7 +3540,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, throw new AlreadyBeingCreatedException(message); case UNDER_CONSTRUCTION: case UNDER_RECOVERY: - final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock; + final BlockInfoContiguousUnderConstruction uc = + (BlockInfoContiguousUnderConstruction)lastBlock; // determine if last block was intended to be truncated Block recoveryBlock = uc.getTruncateBlock(); boolean truncateRecovery = recoveryBlock != null; @@ -3650,9 +3651,8 @@ void finalizeINodeFileUnderConstruction( blockManager.checkReplication(pendingFile); } - @VisibleForTesting - BlockInfoContiguous getStoredBlock(Block block) { - return blockManager.getStoredBlock(block); + public BlockInfoContiguous getStoredBlock(Block block) { + return (BlockInfoContiguous) blockManager.getStoredBlock(block); } @Override @@ -3811,9 +3811,9 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i)); if (storageInfo != null) { if(copyTruncate) { - storageInfo.addBlock(truncatedBlock); + storageInfo.addBlock(truncatedBlock, truncatedBlock); } else { - storageInfo.addBlock(storedBlock); + storageInfo.addBlock(storedBlock, storedBlock); } } } @@ -4163,9 +4163,8 @@ private void clearCorruptLazyPersistFiles() while (it.hasNext()) { Block b = it.next(); - BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b); - if (blockInfo.getBlockCollection().getStoragePolicyID() - == lpPolicy.getId()) { + BlockInfoContiguous blockInfo = getStoredBlock(b); + if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) { filesToDelete.add(blockInfo.getBlockCollection()); } } 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 0daf367362384..c535bd9258774 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 @@ -243,7 +243,8 @@ public void blockIdCK(String blockId) { //get blockInfo Block block = new Block(Block.getBlockId(blockId)); //find which file this block belongs to - BlockInfoContiguous blockInfo = bm.getStoredBlock(block); + BlockInfoContiguous blockInfo = namenode.getNamesystem() + .getStoredBlock(block); if(blockInfo == null) { out.println("Block "+ blockId +" " + NONEXISTENT_STATUS); LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS); 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 c4cbbc178b157..87b370a9a7d43 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 @@ -239,10 +239,12 @@ private void loadFileDiffList(InputStream in, INodeFile file, int size) FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, pbf.getFileSize()); List bpl = pbf.getBlocksList(); + // TODO: also persist striped blocks BlockInfoContiguous[] blocks = new BlockInfoContiguous[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); + BlockInfoContiguous storedBlock = + (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk); if(storedBlock == null) { storedBlock = fsn.getBlockManager().addBlockCollection( new BlockInfoContiguous(blk, copy.getFileReplication()), file); 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 a88a459233751..207d1bba8a3be 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 @@ -1608,8 +1608,8 @@ public static void createKey(String keyName, MiniDFSCluster cluster, */ public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn, ExtendedBlock blk) { - BlockManager bm0 = nn.getNamesystem().getBlockManager(); - BlockInfoContiguous storedBlock = bm0.getStoredBlock(blk.getLocalBlock()); + FSNamesystem fsn = nn.getNamesystem(); + BlockInfoContiguous storedBlock = fsn.getStoredBlock(blk.getLocalBlock()); assertTrue("Block " + blk + " should be under construction, " + "got: " + storedBlock, storedBlock instanceof BlockInfoContiguousUnderConstruction); 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 c5662caabaf55..0500d347be3cf 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 @@ -63,7 +63,7 @@ public void testAddStorage() throws Exception { final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1"); - boolean added = blockInfo.addStorage(storage); + boolean added = blockInfo.addStorage(storage, blockInfo); Assert.assertTrue(added); Assert.assertEquals(storage, blockInfo.getStorageInfo(0)); @@ -129,7 +129,7 @@ public void testBlockListMoveToHead() throws Exception { // 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 void testBlockListMoveToHead() throws Exception { // 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java new file mode 100644 index 0000000000000..74ddac08a2b7c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -0,0 +1,219 @@ +/** + * 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 org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.internal.util.reflection.Whitebox; + +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; + +/** + * Test {@link BlockInfoStriped} + */ +public class TestBlockInfoStriped { + private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + private static final long BASE_ID = -1600; + private static final Block baseBlock = new Block(BASE_ID); + private BlockInfoStriped info; + + @Before + public void setup() { + info = new BlockInfoStriped(baseBlock, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + } + + private Block[] createReportedBlocks(int num) { + Block[] blocks = new Block[num]; + for (int i = 0; i < num; i++) { + blocks[i] = new Block(BASE_ID + i); + } + return blocks; + } + + /** + * Test adding storage and reported block + */ + @Test + public void testAddStorage() { + // first add NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS storages, i.e., a complete + // group of blocks/storages + DatanodeStorageInfo[] storageInfos = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS); + Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS); + int i = 0; + for (; i < storageInfos.length; i += 2) { + info.addStorage(storageInfos[i], blocks[i]); + Assert.assertEquals(i/2 + 1, info.numNodes()); + } + i /= 2; + for (int j = 1; j < storageInfos.length; j += 2) { + Assert.assertTrue(info.addStorage(storageInfos[j], blocks[j])); + Assert.assertEquals(i + (j+1)/2, info.numNodes()); + } + + // check + byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity()); + Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length); + i = 0; + for (DatanodeStorageInfo storage : storageInfos) { + int index = info.findStorageInfo(storage); + Assert.assertEquals(i++, index); + Assert.assertEquals(index, indices[index]); + } + + // the same block is reported from the same storage twice + i = 0; + for (DatanodeStorageInfo storage : storageInfos) { + Assert.assertTrue(info.addStorage(storage, blocks[i++])); + } + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity()); + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.numNodes()); + Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length); + i = 0; + for (DatanodeStorageInfo storage : storageInfos) { + int index = info.findStorageInfo(storage); + Assert.assertEquals(i++, index); + Assert.assertEquals(index, indices[index]); + } + + // the same block is reported from another storage + DatanodeStorageInfo[] storageInfos2 = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS * 2); + // only add the second half of info2 + for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) { + info.addStorage(storageInfos2[i], blocks[i % TOTAL_NUM_BLOCKS]); + Assert.assertEquals(i + 1, info.getCapacity()); + Assert.assertEquals(i + 1, info.numNodes()); + indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(i + 1, indices.length); + } + for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) { + int index = info.findStorageInfo(storageInfos2[i]); + Assert.assertEquals(i++, index); + Assert.assertEquals(index - TOTAL_NUM_BLOCKS, indices[index]); + } + } + + @Test + public void testRemoveStorage() { + // first add TOTAL_NUM_BLOCKS into the BlockInfoStriped + DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS); + Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS); + for (int i = 0; i < storages.length; i++) { + info.addStorage(storages[i], blocks[i]); + } + + // remove two storages + info.removeStorage(storages[0]); + info.removeStorage(storages[2]); + + // check + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity()); + Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes()); + byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices"); + for (int i = 0; i < storages.length; i++) { + int index = info.findStorageInfo(storages[i]); + if (i != 0 && i != 2) { + Assert.assertEquals(i, index); + Assert.assertEquals(index, indices[index]); + } else { + Assert.assertEquals(-1, index); + Assert.assertEquals(-1, indices[i]); + } + } + + // the same block is reported from another storage + DatanodeStorageInfo[] storages2 = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS * 2); + for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) { + info.addStorage(storages2[i], blocks[i % TOTAL_NUM_BLOCKS]); + } + // now we should have 8 storages + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.numNodes()); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity()); + indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length); + int j = TOTAL_NUM_BLOCKS; + for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) { + int index = info.findStorageInfo(storages2[i]); + if (i == TOTAL_NUM_BLOCKS || i == TOTAL_NUM_BLOCKS + 2) { + Assert.assertEquals(i - TOTAL_NUM_BLOCKS, index); + } else { + Assert.assertEquals(j++, index); + } + } + + // remove the storages from storages2 + for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) { + info.removeStorage(storages2[i + TOTAL_NUM_BLOCKS]); + } + // now we should have 3 storages + Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes()); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity()); + indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length); + for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) { + if (i == 0 || i == 2) { + int index = info.findStorageInfo(storages2[i + TOTAL_NUM_BLOCKS]); + Assert.assertEquals(-1, index); + } else { + int index = info.findStorageInfo(storages[i]); + Assert.assertEquals(i, index); + } + } + for (int i = TOTAL_NUM_BLOCKS; i < TOTAL_NUM_BLOCKS * 2 - 2; i++) { + Assert.assertEquals(-1, indices[i]); + Assert.assertNull(info.getDatanode(i)); + } + } + + @Test + public void testReplaceBlock() { + DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS); + Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS); + // add block/storage 0, 2, 4 into the BlockInfoStriped + for (int i = 0; i < storages.length; i += 2) { + Assert.assertEquals(AddBlockResult.ADDED, + storages[i].addBlock(info, blocks[i])); + } + + BlockInfoStriped newBlockInfo = new BlockInfoStriped(info); + info.replaceBlock(newBlockInfo); + + // make sure the newBlockInfo is correct + byte[] indices = (byte[]) Whitebox.getInternalState(newBlockInfo, "indices"); + for (int i = 0; i < storages.length; i += 2) { + int index = newBlockInfo.findStorageInfo(storages[i]); + Assert.assertEquals(i, index); + Assert.assertEquals(index, indices[i]); + + // make sure the newBlockInfo is added to the linked list of the storage + Assert.assertSame(newBlockInfo, storages[i].getBlockListHeadForTesting()); + Assert.assertEquals(1, storages[i].numBlocks()); + Assert.assertNull(newBlockInfo.getNext()); + } + } +} 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 58210c1ffb99b..4612797ab9efc 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 @@ -383,7 +383,7 @@ private void fulfillPipeline(BlockInfoContiguous blockInfo, for (int i = 1; i < pipeline.length; i++) { DatanodeStorageInfo storage = pipeline[i]; bm.addBlock(storage, blockInfo, null); - blockInfo.addStorage(storage); + blockInfo.addStorage(storage, blockInfo); } } @@ -393,7 +393,7 @@ private BlockInfoContiguous blockOnNodes(long blkId, List no for (DatanodeDescriptor dn : nodes) { for (DatanodeStorageInfo storage : dn.getStorageInfos()) { - blockInfo.addStorage(storage); + blockInfo.addStorage(storage, blockInfo); } } return blockInfo; 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 3226578a5b171..2834aadbe0dc5 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 @@ -1244,7 +1244,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() when(storage.removeBlock(any(BlockInfoContiguous.class))).thenReturn(true); when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn (DatanodeStorageInfo.AddBlockResult.ADDED); - ucBlock.addStorage(storage); + ucBlock.addStorage(storage, ucBlock); when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any())) .thenReturn(ucBlock); From dae27f6dd14ac3ed0b9821a3c5239569b13f6adf Mon Sep 17 00:00:00 2001 From: drankye Date: Thu, 12 Feb 2015 21:12:44 +0800 Subject: [PATCH 009/212] HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 4 + .../erasurecode/rawcoder/JRSRawDecoder.java | 69 +++ .../erasurecode/rawcoder/JRSRawEncoder.java | 78 +++ .../erasurecode/rawcoder/RawErasureCoder.java | 2 +- .../rawcoder/util/GaloisField.java | 497 ++++++++++++++++++ .../io/erasurecode/rawcoder/util/RSUtil.java | 22 + .../hadoop/io/erasurecode/TestCoderBase.java | 28 +- .../erasurecode/rawcoder/TestJRSRawCoder.java | 93 ++++ .../rawcoder/TestRawCoderBase.java | 5 +- .../erasurecode/rawcoder/TestXorRawCoder.java | 1 - 10 files changed, 786 insertions(+), 13 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 9728f977bba70..7bbacf7a4e7e9 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -8,3 +8,7 @@ HADOOP-11541. Raw XOR coder ( Kai Zheng ) + + HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng + ( Kai Zheng ) + diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java new file mode 100644 index 0000000000000..dbb689eec752c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java @@ -0,0 +1,69 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; + +import java.nio.ByteBuffer; + +/** + * A raw erasure decoder in RS code scheme in pure Java in case native one + * isn't available in some environment. Please always use native implementations + * when possible. + */ +public class JRSRawDecoder extends AbstractRawErasureDecoder { + // To describe and calculate the needed Vandermonde matrix + private int[] errSignature; + private int[] primitivePower; + + @Override + public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { + super.initialize(numDataUnits, numParityUnits, chunkSize); + assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); + + this.errSignature = new int[getNumParityUnits()]; + this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(), + getNumParityUnits()); + } + + @Override + protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + for (int i = 0; i < erasedIndexes.length; i++) { + errSignature[i] = primitivePower[erasedIndexes[i]]; + RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]); + } + + int dataLen = inputs[0].remaining(); + RSUtil.GF.solveVandermondeSystem(errSignature, outputs, + erasedIndexes.length, dataLen); + } + + @Override + protected void doDecode(byte[][] inputs, int[] erasedIndexes, + byte[][] outputs) { + for (int i = 0; i < erasedIndexes.length; i++) { + errSignature[i] = primitivePower[erasedIndexes[i]]; + RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]); + } + + int dataLen = inputs[0].length; + RSUtil.GF.solveVandermondeSystem(errSignature, outputs, + erasedIndexes.length, dataLen); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java new file mode 100644 index 0000000000000..6ea7551f26e23 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java @@ -0,0 +1,78 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; + +import java.nio.ByteBuffer; + +/** + * A raw erasure encoder in RS code scheme in pure Java in case native one + * isn't available in some environment. Please always use native implementations + * when possible. + */ +public class JRSRawEncoder extends AbstractRawErasureEncoder { + private int[] generatingPolynomial; + + @Override + public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { + super.initialize(numDataUnits, numParityUnits, chunkSize); + assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); + + int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(), + getNumParityUnits()); + // compute generating polynomial + int[] gen = {1}; + int[] poly = new int[2]; + for (int i = 0; i < getNumParityUnits(); i++) { + poly[0] = primitivePower[i]; + poly[1] = 1; + gen = RSUtil.GF.multiply(gen, poly); + } + // generating polynomial has all generating roots + generatingPolynomial = gen; + } + + @Override + protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { + ByteBuffer[] data = new ByteBuffer[getNumDataUnits() + getNumParityUnits()]; + for (int i = 0; i < getNumParityUnits(); i++) { + data[i] = outputs[i]; + } + for (int i = 0; i < getNumDataUnits(); i++) { + data[i + getNumParityUnits()] = inputs[i]; + } + + // Compute the remainder + RSUtil.GF.remainder(data, generatingPolynomial); + } + + @Override + protected void doEncode(byte[][] inputs, byte[][] outputs) { + byte[][] data = new byte[getNumDataUnits() + getNumParityUnits()][]; + for (int i = 0; i < getNumParityUnits(); i++) { + data[i] = outputs[i]; + } + for (int i = 0; i < getNumDataUnits(); i++) { + data[i + getNumParityUnits()] = inputs[i]; + } + + // Compute the remainder + RSUtil.GF.remainder(data, generatingPolynomial); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java index 91a9abfe2aad4..6e07cf1c94583 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -71,7 +71,7 @@ public interface RawErasureCoder { public boolean preferNativeBuffer(); /** - * Should be called when release this coder. Good chance to release encoding + * Should be called when release this blockcoder. Good chance to release encoding * or decoding buffers */ public void release(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java new file mode 100644 index 0000000000000..77544c62235db --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java @@ -0,0 +1,497 @@ +/** + * 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.io.erasurecode.rawcoder.util; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +/** + * Implementation of Galois field arithmetic with 2^p elements. The input must + * be unsigned integers. It's ported from HDFS-RAID, slightly adapted. + */ +public class GaloisField { + + // Field size 256 is good for byte based system + private static final int DEFAULT_FIELD_SIZE = 256; + // primitive polynomial 1 + X^2 + X^3 + X^4 + X^8 (substitute 2) + private static final int DEFAULT_PRIMITIVE_POLYNOMIAL = 285; + static private final Map instances = + new HashMap(); + private final int[] logTable; + private final int[] powTable; + private final int[][] mulTable; + private final int[][] divTable; + private final int fieldSize; + private final int primitivePeriod; + private final int primitivePolynomial; + + private GaloisField(int fieldSize, int primitivePolynomial) { + assert fieldSize > 0; + assert primitivePolynomial > 0; + + this.fieldSize = fieldSize; + this.primitivePeriod = fieldSize - 1; + this.primitivePolynomial = primitivePolynomial; + logTable = new int[fieldSize]; + powTable = new int[fieldSize]; + mulTable = new int[fieldSize][fieldSize]; + divTable = new int[fieldSize][fieldSize]; + int value = 1; + for (int pow = 0; pow < fieldSize - 1; pow++) { + powTable[pow] = value; + logTable[value] = pow; + value = value * 2; + if (value >= fieldSize) { + value = value ^ primitivePolynomial; + } + } + // building multiplication table + for (int i = 0; i < fieldSize; i++) { + for (int j = 0; j < fieldSize; j++) { + if (i == 0 || j == 0) { + mulTable[i][j] = 0; + continue; + } + int z = logTable[i] + logTable[j]; + z = z >= primitivePeriod ? z - primitivePeriod : z; + z = powTable[z]; + mulTable[i][j] = z; + } + } + // building division table + for (int i = 0; i < fieldSize; i++) { + for (int j = 1; j < fieldSize; j++) { + if (i == 0) { + divTable[i][j] = 0; + continue; + } + int z = logTable[i] - logTable[j]; + z = z < 0 ? z + primitivePeriod : z; + z = powTable[z]; + divTable[i][j] = z; + } + } + } + + /** + * Get the object performs Galois field arithmetics + * + * @param fieldSize size of the field + * @param primitivePolynomial a primitive polynomial corresponds to the size + */ + public static GaloisField getInstance(int fieldSize, + int primitivePolynomial) { + int key = ((fieldSize << 16) & 0xFFFF0000) + + (primitivePolynomial & 0x0000FFFF); + GaloisField gf; + synchronized (instances) { + gf = instances.get(key); + if (gf == null) { + gf = new GaloisField(fieldSize, primitivePolynomial); + instances.put(key, gf); + } + } + return gf; + } + + /** + * Get the object performs Galois field arithmetic with default setting + */ + public static GaloisField getInstance() { + return getInstance(DEFAULT_FIELD_SIZE, DEFAULT_PRIMITIVE_POLYNOMIAL); + } + + /** + * Return number of elements in the field + * + * @return number of elements in the field + */ + public int getFieldSize() { + return fieldSize; + } + + /** + * Return the primitive polynomial in GF(2) + * + * @return primitive polynomial as a integer + */ + public int getPrimitivePolynomial() { + return primitivePolynomial; + } + + /** + * Compute the sum of two fields + * + * @param x input field + * @param y input field + * @return result of addition + */ + public int add(int x, int y) { + assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize()); + return x ^ y; + } + + /** + * Compute the multiplication of two fields + * + * @param x input field + * @param y input field + * @return result of multiplication + */ + public int multiply(int x, int y) { + assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize()); + return mulTable[x][y]; + } + + /** + * Compute the division of two fields + * + * @param x input field + * @param y input field + * @return x/y + */ + public int divide(int x, int y) { + assert (x >= 0 && x < getFieldSize() && y > 0 && y < getFieldSize()); + return divTable[x][y]; + } + + /** + * Compute power n of a field + * + * @param x input field + * @param n power + * @return x^n + */ + public int power(int x, int n) { + assert (x >= 0 && x < getFieldSize()); + if (n == 0) { + return 1; + } + if (x == 0) { + return 0; + } + x = logTable[x] * n; + if (x < primitivePeriod) { + return powTable[x]; + } + x = x % primitivePeriod; + return powTable[x]; + } + + /** + * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such + * that Vz=y. The output z will be placed in y. + * + * @param x the vector which describe the Vandermonde matrix + * @param y right-hand side of the Vandermonde system equation. will be + * replaced the output in this vector + */ + public void solveVandermondeSystem(int[] x, int[] y) { + solveVandermondeSystem(x, y, x.length); + } + + /** + * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such + * that Vz=y. The output z will be placed in y. + * + * @param x the vector which describe the Vandermonde matrix + * @param y right-hand side of the Vandermonde system equation. will be + * replaced the output in this vector + * @param len consider x and y only from 0...len-1 + */ + public void solveVandermondeSystem(int[] x, int[] y, int len) { + assert (x.length <= len && y.length <= len); + for (int i = 0; i < len - 1; i++) { + for (int j = len - 1; j > i; j--) { + y[j] = y[j] ^ mulTable[x[i]][y[j - 1]]; + } + } + for (int i = len - 1; i >= 0; i--) { + for (int j = i + 1; j < len; j++) { + y[j] = divTable[y[j]][x[j] ^ x[j - i - 1]]; + } + for (int j = i; j < len - 1; j++) { + y[j] = y[j] ^ y[j + 1]; + } + } + } + + /** + * A "bulk" version to the solving of Vandermonde System + */ + public void solveVandermondeSystem(int[] x, byte[][] y, + int len, int dataLen) { + for (int i = 0; i < len - 1; i++) { + for (int j = len - 1; j > i; j--) { + for (int k = 0; k < dataLen; k++) { + y[j][k] = (byte) (y[j][k] ^ mulTable[x[i]][y[j - 1][k] & + 0x000000FF]); + } + } + } + for (int i = len - 1; i >= 0; i--) { + for (int j = i + 1; j < len; j++) { + for (int k = 0; k < dataLen; k++) { + y[j][k] = (byte) (divTable[y[j][k] & 0x000000FF][x[j] ^ + x[j - i - 1]]); + } + } + for (int j = i; j < len - 1; j++) { + for (int k = 0; k < dataLen; k++) { + y[j][k] = (byte) (y[j][k] ^ y[j + 1][k]); + } + } + } + } + + /** + * A "bulk" version of the solveVandermondeSystem, using ByteBuffer. + */ + public void solveVandermondeSystem(int[] x, ByteBuffer[] y, + int len, int dataLen) { + for (int i = 0; i < len - 1; i++) { + for (int j = len - 1; j > i; j--) { + for (int k = 0; k < dataLen; k++) { + y[j].put(k, (byte) (y[j].get(k) ^ mulTable[x[i]][y[j - 1].get(k) & + 0x000000FF])); + } + } + } + for (int i = len - 1; i >= 0; i--) { + for (int j = i + 1; j < len; j++) { + for (int k = 0; k < dataLen; k++) { + y[j].put(k, (byte) (divTable[y[j].get(k) & 0x000000FF][x[j] ^ + x[j - i - 1]])); + } + } + for (int j = i; j < len - 1; j++) { + for (int k = 0; k < dataLen; k++) { + y[j].put(k, (byte) (y[j].get(k) ^ y[j + 1].get(k))); + } + } + } + } + + /** + * Compute the multiplication of two polynomials. The index in the array + * corresponds to the power of the entry. For example p[0] is the constant + * term of the polynomial p. + * + * @param p input polynomial + * @param q input polynomial + * @return polynomial represents p*q + */ + public int[] multiply(int[] p, int[] q) { + int len = p.length + q.length - 1; + int[] result = new int[len]; + for (int i = 0; i < len; i++) { + result[i] = 0; + } + for (int i = 0; i < p.length; i++) { + + for (int j = 0; j < q.length; j++) { + result[i + j] = add(result[i + j], multiply(p[i], q[j])); + } + } + return result; + } + + /** + * Compute the remainder of a dividend and divisor pair. The index in the + * array corresponds to the power of the entry. For example p[0] is the + * constant term of the polynomial p. + * + * @param dividend dividend polynomial, the remainder will be placed + * here when return + * @param divisor divisor polynomial + */ + public void remainder(int[] dividend, int[] divisor) { + for (int i = dividend.length - divisor.length; i >= 0; i--) { + int ratio = divTable[dividend[i + + divisor.length - 1]][divisor[divisor.length - 1]]; + for (int j = 0; j < divisor.length; j++) { + int k = j + i; + dividend[k] = dividend[k] ^ mulTable[ratio][divisor[j]]; + } + } + } + + /** + * Compute the sum of two polynomials. The index in the array corresponds to + * the power of the entry. For example p[0] is the constant term of the + * polynomial p. + * + * @param p input polynomial + * @param q input polynomial + * @return polynomial represents p+q + */ + public int[] add(int[] p, int[] q) { + int len = Math.max(p.length, q.length); + int[] result = new int[len]; + for (int i = 0; i < len; i++) { + if (i < p.length && i < q.length) { + result[i] = add(p[i], q[i]); + } else if (i < p.length) { + result[i] = p[i]; + } else { + result[i] = q[i]; + } + } + return result; + } + + /** + * Substitute x into polynomial p(x). + * + * @param p input polynomial + * @param x input field + * @return p(x) + */ + public int substitute(int[] p, int x) { + int result = 0; + int y = 1; + for (int i = 0; i < p.length; i++) { + result = result ^ mulTable[p[i]][y]; + y = mulTable[x][y]; + } + return result; + } + + /** + * A "bulk" version of the substitute. + * Tends to be 2X faster than the "int" substitute in a loop. + * + * @param p input polynomial + * @param q store the return result + * @param x input field + */ + public void substitute(byte[][] p, byte[] q, int x) { + int y = 1; + for (int i = 0; i < p.length; i++) { + byte[] pi = p[i]; + for (int j = 0; j < pi.length; j++) { + int pij = pi[j] & 0x000000FF; + q[j] = (byte) (q[j] ^ mulTable[pij][y]); + } + y = mulTable[x][y]; + } + } + + /** + * A "bulk" version of the substitute, using ByteBuffer. + * Tends to be 2X faster than the "int" substitute in a loop. + * + * @param p input polynomial + * @param q store the return result + * @param x input field + */ + public void substitute(ByteBuffer[] p, ByteBuffer q, int x) { + int y = 1; + for (int i = 0; i < p.length; i++) { + ByteBuffer pi = p[i]; + int len = pi.remaining(); + for (int j = 0; j < len; j++) { + int pij = pi.get(j) & 0x000000FF; + q.put(j, (byte) (q.get(j) ^ mulTable[pij][y])); + } + y = mulTable[x][y]; + } + } + + /** + * The "bulk" version of the remainder. + * Warning: This function will modify the "dividend" inputs. + */ + public void remainder(byte[][] dividend, int[] divisor) { + for (int i = dividend.length - divisor.length; i >= 0; i--) { + for (int j = 0; j < divisor.length; j++) { + for (int k = 0; k < dividend[i].length; k++) { + int ratio = divTable[dividend[i + divisor.length - 1][k] & + 0x00FF][divisor[divisor.length - 1]]; + dividend[j + i][k] = (byte) ((dividend[j + i][k] & 0x00FF) ^ + mulTable[ratio][divisor[j]]); + } + } + } + } + + /** + * The "bulk" version of the remainder, using ByteBuffer. + * Warning: This function will modify the "dividend" inputs. + */ + public void remainder(ByteBuffer[] dividend, int[] divisor) { + for (int i = dividend.length - divisor.length; i >= 0; i--) { + int width = dividend[i].remaining(); + for (int j = 0; j < divisor.length; j++) { + for (int k = 0; k < width; k++) { + int ratio = divTable[dividend[i + divisor.length - 1].get(k) & + 0x00FF][divisor[divisor.length - 1]]; + dividend[j + i].put(k, (byte) ((dividend[j + i].get(k) & 0x00FF) ^ + mulTable[ratio][divisor[j]])); + } + } + } + } + + /** + * Perform Gaussian elimination on the given matrix. This matrix has to be a + * fat matrix (number of rows > number of columns). + */ + public void gaussianElimination(int[][] matrix) { + assert(matrix != null && matrix.length > 0 && matrix[0].length > 0 + && matrix.length < matrix[0].length); + int height = matrix.length; + int width = matrix[0].length; + for (int i = 0; i < height; i++) { + boolean pivotFound = false; + // scan the column for a nonzero pivot and swap it to the diagonal + for (int j = i; j < height; j++) { + if (matrix[i][j] != 0) { + int[] tmp = matrix[i]; + matrix[i] = matrix[j]; + matrix[j] = tmp; + pivotFound = true; + break; + } + } + if (!pivotFound) { + continue; + } + int pivot = matrix[i][i]; + for (int j = i; j < width; j++) { + matrix[i][j] = divide(matrix[i][j], pivot); + } + for (int j = i + 1; j < height; j++) { + int lead = matrix[j][i]; + for (int k = i; k < width; k++) { + matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k])); + } + } + } + for (int i = height - 1; i >=0; i--) { + for (int j = 0; j < i; j++) { + int lead = matrix[j][i]; + for (int k = i; k < width; k++) { + matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k])); + } + } + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java new file mode 100644 index 0000000000000..33ba5614a3abf --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java @@ -0,0 +1,22 @@ +package org.apache.hadoop.io.erasurecode.rawcoder.util; + +/** + * Some utilities for Reed-Solomon coding. + */ +public class RSUtil { + + // We always use the byte system (with symbol size 8, field size 256, + // primitive polynomial 285, and primitive root 2). + public static GaloisField GF = GaloisField.getInstance(); + public static final int PRIMITIVE_ROOT = 2; + + public static int[] getPrimitivePower(int numDataUnits, int numParityUnits) { + int[] primitivePower = new int[numDataUnits + numParityUnits]; + // compute powers of the primitive root + for (int i = 0; i < numDataUnits + numParityUnits; i++) { + primitivePower[i] = GF.power(PRIMITIVE_ROOT, i); + } + return primitivePower; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 9482b4348254c..3c4288c8c5990 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -18,9 +18,11 @@ package org.apache.hadoop.io.erasurecode; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Random; import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertTrue; /** * Test base of common utilities for tests not only raw coders but also block @@ -41,6 +43,14 @@ public abstract class TestCoderBase { // may go to different coding implementations. protected boolean usingDirectBuffer = true; + protected void prepare(int numDataUnits, int numParityUnits, + int[] erasedIndexes) { + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + this.erasedDataIndexes = erasedIndexes != null ? + erasedIndexes : new int[] {0}; + } + /** * Compare and verify if erased chunks are equal to recovered chunks * @param erasedChunks @@ -50,10 +60,8 @@ protected void compareAndVerify(ECChunk[] erasedChunks, ECChunk[] recoveredChunks) { byte[][] erased = ECChunk.toArray(erasedChunks); byte[][] recovered = ECChunk.toArray(recoveredChunks); - for (int i = 0; i < erasedChunks.length; ++i) { - assertArrayEquals("Decoding and comparing failed.", erased[i], - recovered[i]); - } + boolean result = Arrays.deepEquals(erased, recovered); + assertTrue("Decoding and comparing failed.", result); } /** @@ -63,7 +71,7 @@ protected void compareAndVerify(ECChunk[] erasedChunks, */ protected int[] getErasedIndexesForDecoding() { int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length]; - for (int i = 0; i < erasedDataIndexes.length; ++i) { + for (int i = 0; i < erasedDataIndexes.length; i++) { erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits; } return erasedIndexesForDecoding; @@ -100,7 +108,7 @@ protected ECChunk[] copyDataChunksToErase(ECChunk[] dataChunks) { ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length]; int j = 0; - for (int i = 0; i < erasedDataIndexes.length; ++i) { + for (int i = 0; i < erasedDataIndexes.length; i++) { copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]); } @@ -112,7 +120,7 @@ protected ECChunk[] copyDataChunksToErase(ECChunk[] dataChunks) { * @param dataChunks */ protected void eraseSomeDataBlocks(ECChunk[] dataChunks) { - for (int i = 0; i < erasedDataIndexes.length; ++i) { + for (int i = 0; i < erasedDataIndexes.length; i++) { eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]); } } @@ -122,7 +130,7 @@ protected void eraseSomeDataBlocks(ECChunk[] dataChunks) { * @param chunks */ protected void eraseDataFromChunks(ECChunk[] chunks) { - for (int i = 0; i < chunks.length; ++i) { + for (int i = 0; i < chunks.length; i++) { eraseDataFromChunk(chunks[i]); } } @@ -135,7 +143,7 @@ protected void eraseDataFromChunk(ECChunk chunk) { ByteBuffer chunkBuffer = chunk.getBuffer(); // erase the data chunkBuffer.position(0); - for (int i = 0; i < chunkSize; ++i) { + for (int i = 0; i < chunkSize; i++) { chunkBuffer.put((byte) 0); } chunkBuffer.flip(); @@ -150,7 +158,7 @@ protected void eraseDataFromChunk(ECChunk chunk) { */ protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) { ECChunk[] results = new ECChunk[chunks.length]; - for (int i = 0; i < chunks.length; ++i) { + for (int i = 0; i < chunks.length; i++) { results[i] = cloneChunkWithData(chunks[i]); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java new file mode 100644 index 0000000000000..e54f647cf6c8f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java @@ -0,0 +1,93 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; +import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + * Test raw Reed-solomon encoding and decoding. + */ +public class TestJRSRawCoder extends TestRawCoderBase { + + private static int symbolSize = 0; + private static int symbolMax = 0; + + static { + symbolSize = (int) Math.round(Math.log( + RSUtil.GF.getFieldSize()) / Math.log(2)); + symbolMax = (int) Math.pow(2, symbolSize); + } + + @Before + public void setup() { + this.encoderClass = JRSRawEncoder.class; + this.decoderClass = JRSRawDecoder.class; + } + + @Test + public void testCodingNoDirectBuffer_10x4() { + prepare(10, 4, null); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_10x4() { + prepare(10, 4, null); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasure_of_2_4() { + prepare(10, 4, new int[] {2, 4}); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_all() { + prepare(10, 4, new int[] {0, 1, 2, 3}); + testCoding(true); + } + + @Test + public void testCodingNoDirectBuffer_3x3() { + prepare(3, 3, null); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_3x3() { + prepare(3, 3, null); + testCoding(true); + } + + @Override + protected ECChunk generateDataChunk() { + ByteBuffer buffer = allocateOutputBuffer(); + for (int i = 0; i < chunkSize; i++) { + buffer.put((byte) RAND.nextInt(symbolMax)); + } + buffer.flip(); + + return new ECChunk(buffer); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 9119211641fa4..5f6ccda23893b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -31,10 +31,13 @@ public abstract class TestRawCoderBase extends TestCoderBase { * Generating source data, encoding, recovering and then verifying. * RawErasureCoder mainly uses ECChunk to pass input and output data buffers, * it supports two kinds of ByteBuffers, one is array backed, the other is - * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test. + * direct ByteBuffer. Use usingDirectBuffer indicate which case to test. + * * @param usingDirectBuffer */ protected void testCoding(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + // Generate data and encode ECChunk[] dataChunks = prepareDataChunksForEncoding(); ECChunk[] parityChunks = prepareParityChunksForEncoding(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java index 8e59b8a2c3ca7..ff4858616d4f3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java @@ -26,7 +26,6 @@ * Test XOR encoding and decoding. */ public class TestXorRawCoder extends TestRawCoderBase { - private static Random RAND = new Random(); @Before public void setup() { From 9f2f583f401189c3f4a2687795a9e3e0b288322b Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 25 Feb 2015 22:10:26 -0800 Subject: [PATCH 010/212] HDFS-7749. Erasure Coding: Add striped block support in INodeFile. Contributed by Jing Zhao. --- .../hadoop/hdfs/protocolPB/PBHelper.java | 17 ++ .../blockmanagement/BlockCollection.java | 13 +- .../server/blockmanagement/BlockInfo.java | 88 +++++- .../BlockInfoContiguousUnderConstruction.java | 6 +- .../blockmanagement/BlockInfoStriped.java | 31 ++ .../BlockInfoStripedUnderConstruction.java | 240 ++++++++++++++++ .../server/blockmanagement/BlockManager.java | 148 ++++------ .../CacheReplicationMonitor.java | 16 +- .../hdfs/server/namenode/FSDirConcatOp.java | 8 +- .../hdfs/server/namenode/FSDirectory.java | 5 +- .../hdfs/server/namenode/FSEditLog.java | 8 +- .../hdfs/server/namenode/FSEditLogLoader.java | 16 +- .../hdfs/server/namenode/FSImageFormat.java | 7 +- .../server/namenode/FSImageFormatPBINode.java | 46 ++- .../hdfs/server/namenode/FSNamesystem.java | 91 +++--- .../FileUnderConstructionFeature.java | 15 +- .../FileWithStripedBlocksFeature.java | 112 ++++++++ .../hdfs/server/namenode/INodeFile.java | 266 +++++++++++++----- .../hdfs/server/namenode/LeaseManager.java | 8 +- .../hdfs/server/namenode/NamenodeFsck.java | 4 +- .../hdfs/server/namenode/Namesystem.java | 3 +- .../snapshot/FSImageFormatPBSnapshot.java | 7 +- .../namenode/snapshot/FileDiffList.java | 6 +- .../hadoop-hdfs/src/main/proto/fsimage.proto | 5 + .../hadoop-hdfs/src/main/proto/hdfs.proto | 10 + .../org/apache/hadoop/hdfs/DFSTestUtil.java | 3 +- .../TestReplicationPolicy.java | 4 +- .../hdfs/server/namenode/TestAddBlock.java | 12 +- .../server/namenode/TestAddBlockgroup.java | 3 +- .../namenode/TestBlockUnderConstruction.java | 6 +- .../hdfs/server/namenode/TestFSImage.java | 4 +- .../server/namenode/TestFileTruncate.java | 4 +- .../hadoop/hdfs/server/namenode/TestFsck.java | 4 +- .../snapshot/TestSnapshotBlocksMap.java | 24 +- .../snapshot/TestSnapshotDeletion.java | 16 +- 35 files changed, 966 insertions(+), 290 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java 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 c9a9c337a0238..3fb5fc6e9f6c9 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 @@ -172,6 +172,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos; import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto; @@ -184,6 +185,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; @@ -428,6 +430,21 @@ public static Block convert(BlockProto b) { return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp()); } + public static BlockInfoStriped convert(StripedBlockProto p) { + return new BlockInfoStriped(convert(p.getBlock()), + (short) p.getDataBlockNum(), (short) p.getParityBlockNum()); + } + + public static StripedBlockProto convert(BlockInfoStriped blk) { + BlockProto bp = BlockProto.newBuilder().setBlockId(blk.getBlockId()) + .setGenStamp(blk.getGenerationStamp()).setNumBytes(blk.getNumBytes()) + .build(); + return StripedBlockProto.newBuilder() + .setDataBlockNum(blk.getDataBlockNum()) + .setParityBlockNum(blk.getParityBlockNum()) + .setBlock(bp).build(); + } + public static BlockWithLocationsProto convert(BlockWithLocations blk) { return BlockWithLocationsProto.newBuilder() .setBlock(convert(blk.getBlock())) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java index e873946a52006..250bdcd5ecf16 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java @@ -31,7 +31,7 @@ public interface BlockCollection { /** * Get the last block of the collection. */ - public BlockInfoContiguous getLastBlock(); + public BlockInfo getLastBlock(); /** * Get content summary. @@ -44,9 +44,9 @@ public interface BlockCollection { public int numBlocks(); /** - * Get the blocks or block groups. + * Get the blocks (striped or contiguous). */ - public BlockInfoContiguous[] getBlocks(); + public BlockInfo[] getBlocks(); /** * Get preferred block size for the collection @@ -71,16 +71,15 @@ public interface BlockCollection { public String getName(); /** - * Set the block/block-group at the given index. + * Set the block (contiguous or striped) at the given index. */ - public void setBlock(int index, BlockInfoContiguous blk); + public void setBlock(int index, BlockInfo blk); /** * Convert the last block of the collection to an under-construction block * and set the locations. */ - public BlockInfoContiguousUnderConstruction setLastBlock( - BlockInfoContiguous lastBlock, + public void convertLastBlockToUC(BlockInfo lastBlock, DatanodeStorageInfo[] targets) throws IOException; /** 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 d3ea813b2eb63..bf8e64f4ed7e1 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 org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.util.LightWeightGSet; +import java.io.IOException; import java.util.LinkedList; /** @@ -289,8 +290,9 @@ public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage, /** * BlockInfo represents a block that is not being constructed. - * In order to start modifying the block, the BlockInfo should be converted - * to {@link BlockInfoContiguousUnderConstruction}. + * In order to start modifying the block, the BlockInfo should be converted to + * {@link BlockInfoContiguousUnderConstruction} or + * {@link BlockInfoStripedUnderConstruction}. * @return {@link HdfsServerConstants.BlockUCState#COMPLETE} */ public HdfsServerConstants.BlockUCState getBlockUCState() { @@ -340,4 +342,86 @@ static BlockInfo copyOf(BlockInfo b) { return new BlockInfoStriped((BlockInfoStriped) b); } } + + static BlockInfo convertToCompleteBlock(BlockInfo blk) throws IOException { + if (blk instanceof BlockInfoContiguousUnderConstruction) { + return ((BlockInfoContiguousUnderConstruction) blk) + .convertToCompleteBlock(); + } else if (blk instanceof BlockInfoStripedUnderConstruction) { + return ((BlockInfoStripedUnderConstruction) blk).convertToCompleteBlock(); + } else { + return blk; + } + } + + static void commitBlock(BlockInfo blockInfo, Block reported) + throws IOException { + if (blockInfo instanceof BlockInfoContiguousUnderConstruction) { + ((BlockInfoContiguousUnderConstruction) blockInfo).commitBlock(reported); + } else if (blockInfo instanceof BlockInfoStripedUnderConstruction) { + ((BlockInfoStripedUnderConstruction) blockInfo).commitBlock(reported); + } + } + + static void addReplica(BlockInfo ucBlock, DatanodeStorageInfo storageInfo, + Block reportedBlock, HdfsServerConstants.ReplicaState reportedState) { + assert ucBlock instanceof BlockInfoContiguousUnderConstruction || + ucBlock instanceof BlockInfoStripedUnderConstruction; + if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { + ((BlockInfoContiguousUnderConstruction) ucBlock).addReplicaIfNotPresent( + storageInfo, reportedBlock, reportedState); + } else { // StripedUC + ((BlockInfoStripedUnderConstruction) ucBlock).addReplicaIfNotPresent( + storageInfo, reportedBlock, reportedState); + } + } + + static int getNumExpectedLocations(BlockInfo ucBlock) { + assert ucBlock instanceof BlockInfoContiguousUnderConstruction || + ucBlock instanceof BlockInfoStripedUnderConstruction; + if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { + return ((BlockInfoContiguousUnderConstruction) ucBlock) + .getNumExpectedLocations(); + } else { // StripedUC + return ((BlockInfoStripedUnderConstruction) ucBlock) + .getNumExpectedLocations(); + } + } + + public static DatanodeStorageInfo[] getExpectedStorageLocations( + BlockInfo ucBlock) { + assert ucBlock instanceof BlockInfoContiguousUnderConstruction || + ucBlock instanceof BlockInfoStripedUnderConstruction; + if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { + return ((BlockInfoContiguousUnderConstruction) ucBlock) + .getExpectedStorageLocations(); + } else { // StripedUC + return ((BlockInfoStripedUnderConstruction) ucBlock) + .getExpectedStorageLocations(); + } + } + + public static void setExpectedLocations(BlockInfo ucBlock, + DatanodeStorageInfo[] targets) { + assert ucBlock instanceof BlockInfoContiguousUnderConstruction || + ucBlock instanceof BlockInfoStripedUnderConstruction; + if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { + ((BlockInfoContiguousUnderConstruction) ucBlock) + .setExpectedLocations(targets); + } else { // StripedUC + ((BlockInfoStripedUnderConstruction) ucBlock) + .setExpectedLocations(targets); + } + } + + public static long getBlockRecoveryId(BlockInfo ucBlock) { + assert ucBlock instanceof BlockInfoContiguousUnderConstruction || + ucBlock instanceof BlockInfoStripedUnderConstruction; + if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { + return ((BlockInfoContiguousUnderConstruction) ucBlock) + .getBlockRecoveryId(); + } else { // StripedUC + return ((BlockInfoStripedUnderConstruction) ucBlock).getBlockRecoveryId(); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java index c78c9e2c45558..7a052fd175fa1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java @@ -74,7 +74,7 @@ public BlockInfoContiguousUnderConstruction(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) { super(blk, replication); assert getBlockUCState() != BlockUCState.COMPLETE : - "BlockInfoUnderConstruction cannot be in COMPLETE state"; + "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state"; this.blockUCState = state; setExpectedLocations(targets); } @@ -82,7 +82,7 @@ assert getBlockUCState() != BlockUCState.COMPLETE : /** * Convert an under construction block to a complete block. * - * @return BlockInfo - a complete block. + * @return BlockInfoContiguous - a complete block. * @throws IOException if the state of the block * (the generation stamp and the length) has not been committed by * the client or it does not have at least a minimal number of replicas @@ -197,7 +197,7 @@ public void initializeBlockRecovery(long recoveryId) { blockRecoveryId = recoveryId; if (replicas.size() == 0) { NameNode.blockStateChangeLog.warn("BLOCK*" - + " BlockInfoUnderConstruction.initLeaseRecovery:" + + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:" + " No blocks found, lease removed."); } boolean allLiveReplicasTriedAsPrimary = true; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 5fff41e59f7bd..57de772f11f9b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; /** * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. @@ -59,6 +61,14 @@ private short getTotalBlockNum() { return (short) (dataBlockNum + parityBlockNum); } + public short getDataBlockNum() { + return dataBlockNum; + } + + public short getParityBlockNum() { + return parityBlockNum; + } + private void initIndices() { for (int i = 0; i < indices.length; i++) { indices[i] = -1; @@ -176,4 +186,25 @@ public int numNodes() { } return num; } + + /** + * Convert a complete block to an under construction block. + * @return BlockInfoUnderConstruction - an under construction block. + */ + public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( + BlockUCState s, DatanodeStorageInfo[] targets) { + final BlockInfoStripedUnderConstruction ucBlock; + if(isComplete()) { + ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(), + getParityBlockNum(), s, targets); + ucBlock.setBlockCollection(getBlockCollection()); + } else { + // the block is already under construction + ucBlock = (BlockInfoStripedUnderConstruction) this; + ucBlock.setBlockUCState(s); + ucBlock.setExpectedLocations(targets); + ucBlock.setBlockCollection(getBlockCollection()); + } + return ucBlock; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java new file mode 100644 index 0000000000000..151241b25efde --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -0,0 +1,240 @@ +/** + * 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 org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION; + +/** + * Represents a striped block that is currently being constructed. + * This is usually the last block of a file opened for write or append. + */ +public class BlockInfoStripedUnderConstruction extends BlockInfoStriped { + private BlockUCState blockUCState; + + /** + * Block replicas as assigned when the block was allocated. + * + * TODO: we need to update this attribute, along with the return type of + * getExpectedStorageLocations and LocatedBlock. For striped blocks, clients + * need to understand the index of each striped block in the block group. + */ + private List replicas; + + /** + * The new generation stamp, which this block will have + * after the recovery succeeds. Also used as a recovery id to identify + * the right recovery if any of the abandoned recoveries re-appear. + */ + private long blockRecoveryId = 0; + + /** + * Constructor with null storage targets. + */ + public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum, + short parityBlockNum) { + this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null); + } + + /** + * Create a striped block that is currently being constructed. + */ + public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum, + short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) { + super(blk, dataBlockNum, parityBlockNum); + assert getBlockUCState() != COMPLETE : + "BlockInfoStripedUnderConstruction cannot be in COMPLETE state"; + this.blockUCState = state; + setExpectedLocations(targets); + } + + /** + * Convert an under construction striped block to a complete striped block. + * + * @return BlockInfoStriped - a complete block. + * @throws IOException if the state of the block + * (the generation stamp and the length) has not been committed by + * the client or it does not have at least a minimal number of replicas + * reported from data-nodes. + */ + BlockInfoStriped convertToCompleteBlock() throws IOException { + assert getBlockUCState() != COMPLETE : + "Trying to convert a COMPLETE block"; + return new BlockInfoStriped(this); + } + + /** Set expected locations */ + public void setExpectedLocations(DatanodeStorageInfo[] targets) { + int numLocations = targets == null ? 0 : targets.length; + this.replicas = new ArrayList<>(numLocations); + for(int i = 0; i < numLocations; i++) { + replicas.add(new ReplicaUnderConstruction(this, targets[i], + ReplicaState.RBW)); + } + } + + /** + * Create array of expected replica locations + * (as has been assigned by chooseTargets()). + */ + public DatanodeStorageInfo[] getExpectedStorageLocations() { + int numLocations = getNumExpectedLocations(); + DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; + for (int i = 0; i < numLocations; i++) { + storages[i] = replicas.get(i).getExpectedStorageLocation(); + } + return storages; + } + + /** Get the number of expected locations */ + public int getNumExpectedLocations() { + return replicas == null ? 0 : replicas.size(); + } + + /** + * Return the state of the block under construction. + * @see BlockUCState + */ + @Override // BlockInfo + public BlockUCState getBlockUCState() { + return blockUCState; + } + + void setBlockUCState(BlockUCState s) { + blockUCState = s; + } + + /** Get block recovery ID */ + public long getBlockRecoveryId() { + return blockRecoveryId; + } + + /** + * Process the recorded replicas. When about to commit or finish the + * pipeline recovery sort out bad replicas. + * @param genStamp The final generation stamp for the block. + */ + public void setGenerationStampAndVerifyReplicas(long genStamp) { + // Set the generation stamp for the block. + setGenerationStamp(genStamp); + if (replicas == null) + return; + + // Remove the replicas with wrong gen stamp. + // The replica list is unchanged. + for (ReplicaUnderConstruction r : replicas) { + if (genStamp != r.getGenerationStamp()) { + r.getExpectedStorageLocation().removeBlock(this); + NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica " + + "from location: {}", r.getExpectedStorageLocation()); + } + } + } + + /** + * Commit block's length and generation stamp as reported by the client. + * Set block state to {@link BlockUCState#COMMITTED}. + * @param block - contains client reported block length and generation + */ + void commitBlock(Block block) throws IOException { + if (getBlockId() != block.getBlockId()) { + throw new IOException("Trying to commit inconsistent block: id = " + + block.getBlockId() + ", expected id = " + getBlockId()); + } + blockUCState = BlockUCState.COMMITTED; + this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp()); + // Sort out invalid replicas. + setGenerationStampAndVerifyReplicas(block.getGenerationStamp()); + } + + /** + * Initialize lease recovery for this striped block. + */ + public void initializeBlockRecovery(long recoveryId) { + setBlockUCState(BlockUCState.UNDER_RECOVERY); + blockRecoveryId = recoveryId; + if (replicas == null || replicas.size() == 0) { + NameNode.blockStateChangeLog.warn("BLOCK*" + + " BlockInfoUnderConstruction.initLeaseRecovery:" + + " No blocks found, lease removed."); + } + // TODO we need to implement different recovery logic here + } + + void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block, + ReplicaState rState) { + Iterator it = replicas.iterator(); + while (it.hasNext()) { + ReplicaUnderConstruction r = it.next(); + DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation(); + if (expectedLocation == storage) { + // Record the gen stamp from the report + r.setGenerationStamp(block.getGenerationStamp()); + return; + } else if (expectedLocation != null && + expectedLocation.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, storage, rState)); + } + + @Override + public String toString() { + final StringBuilder b = new StringBuilder(100); + appendStringTo(b); + return b.toString(); + } + + @Override + public void appendStringTo(StringBuilder sb) { + super.appendStringTo(sb); + appendUCParts(sb); + } + + private void appendUCParts(StringBuilder sb) { + sb.append("{UCState=").append(blockUCState).append(", replicas=["); + if (replicas != null) { + Iterator iter = replicas.iterator(); + if (iter.hasNext()) { + iter.next().appendStringTo(sb); + while (iter.hasNext()) { + sb.append(", "); + iter.next().appendStringTo(sb); + } + } + } + sb.append("]}"); + } +} 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 6c0fb3031d4c3..80394aaf1cf7a 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 @@ -543,8 +543,8 @@ private void dumpBlockMeta(Block block, PrintWriter out) { int usableReplicas = numReplicas.liveReplicas() + numReplicas.decommissionedAndDecommissioning(); - if (block instanceof BlockInfoContiguous) { - BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection(); + if (block instanceof BlockInfo) { + BlockCollection bc = ((BlockInfo) block).getBlockCollection(); String fileName = (bc == null) ? "[orphaned]" : bc.getName(); out.print(fileName + ": "); } @@ -598,15 +598,14 @@ public boolean checkMinReplication(Block block) { * @throws IOException if the block does not have at least a minimal number * of replicas reported from data-nodes. */ - private static boolean commitBlock( - final BlockInfoContiguousUnderConstruction block, + private static boolean commitBlock(final BlockInfo block, final Block commitBlock) throws IOException { if (block.getBlockUCState() == BlockUCState.COMMITTED) return false; assert block.getNumBytes() <= commitBlock.getNumBytes() : "commitBlock length is less than the stored one " + commitBlock.getNumBytes() + " vs. " + block.getNumBytes(); - block.commitBlock(commitBlock); + BlockInfo.commitBlock(block, commitBlock); return true; } @@ -624,16 +623,16 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, Block commitBlock) throws IOException { if(commitBlock == null) return false; // not committing, this is a block allocation retry - BlockInfoContiguous lastBlock = bc.getLastBlock(); + BlockInfo lastBlock = bc.getLastBlock(); if(lastBlock == null) return false; // no blocks in file yet if(lastBlock.isComplete()) return false; // already completed (e.g. by syncBlock) - final boolean b = commitBlock( - (BlockInfoContiguousUnderConstruction)lastBlock, commitBlock); - if(countNodes(lastBlock).liveReplicas() >= minReplication) - completeBlock(bc, bc.numBlocks()-1, false); + final boolean b = commitBlock(lastBlock, commitBlock); + if (countNodes(lastBlock).liveReplicas() >= minReplication) { + completeBlock(bc, bc.numBlocks() - 1, false); + } return b; } @@ -646,22 +645,25 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, */ private BlockInfo completeBlock(final BlockCollection bc, final int blkIndex, boolean force) throws IOException { - if(blkIndex < 0) + if (blkIndex < 0) { return null; - BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex]; - if (curBlock.isComplete()) + } + BlockInfo curBlock = bc.getBlocks()[blkIndex]; + if (curBlock.isComplete()) { return curBlock; - // TODO: support BlockInfoStripedUC - BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction)curBlock; - int numNodes = ucBlock.numNodes(); - if (!force && numNodes < minReplication) + } + + int numNodes = curBlock.numNodes(); + if (!force && numNodes < minReplication) { throw new IOException("Cannot complete block: " + "block does not satisfy minimal replication requirement."); - if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED) + } + if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) { throw new IOException( "Cannot complete block: block has not been COMMITTED by the client"); - BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock(); + } + + final BlockInfo completeBlock = BlockInfo.convertToCompleteBlock(curBlock); // replace penultimate block in file bc.setBlock(blkIndex, completeBlock); @@ -679,10 +681,9 @@ private BlockInfo completeBlock(final BlockCollection bc, return blocksMap.replaceBlock(completeBlock); } - // TODO: support BlockInfoStrippedUC private BlockInfo completeBlock(final BlockCollection bc, final BlockInfo block, boolean force) throws IOException { - BlockInfoContiguous[] fileBlocks = bc.getBlocks(); + BlockInfo[] fileBlocks = bc.getBlocks(); for (int idx = 0; idx < fileBlocks.length; idx++) { if (fileBlocks[idx] == block) { return completeBlock(bc, idx, force); @@ -698,6 +699,7 @@ private BlockInfo completeBlock(final BlockCollection bc, */ public BlockInfo forceCompleteBlock(final BlockCollection bc, final BlockInfoContiguousUnderConstruction block) throws IOException { + // TODO: support BlockInfoStripedUC for editlog block.commitBlock(block); return completeBlock(bc, block, true); } @@ -719,7 +721,7 @@ public BlockInfo forceCompleteBlock(final BlockCollection bc, */ public LocatedBlock convertLastBlockToUnderConstruction( BlockCollection bc, long bytesToRemove) throws IOException { - BlockInfoContiguous oldBlock = bc.getLastBlock(); + BlockInfo oldBlock = bc.getLastBlock(); if(oldBlock == null || bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove) return null; @@ -728,8 +730,10 @@ public LocatedBlock convertLastBlockToUnderConstruction( DatanodeStorageInfo[] targets = getStorages(oldBlock); - BlockInfoContiguousUnderConstruction ucBlock = bc.setLastBlock(oldBlock, - targets); + // convert the last block to UC + bc.convertLastBlockToUC(oldBlock, targets); + // get the new created uc block + BlockInfo ucBlock = bc.getLastBlock(); blocksMap.replaceBlock(ucBlock); // Remove block from replication queue. @@ -771,11 +775,10 @@ private List getValidLocations(Block block) { return locations; } - private List createLocatedBlockList( - final BlockInfoContiguous[] blocks, + private List createLocatedBlockList(final BlockInfo[] blocks, final long offset, final long length, final int nrBlocksToReturn, final AccessMode mode) throws IOException { - int curBlk = 0; + int curBlk; long curPos = 0, blkSize = 0; int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length; for (curBlk = 0; curBlk < nrBlocks; curBlk++) { @@ -788,10 +791,10 @@ private List createLocatedBlockList( } if (nrBlocks > 0 && curBlk == nrBlocks) // offset >= end of file - return Collections.emptyList(); + return Collections.emptyList(); long endOff = offset + length; - List results = new ArrayList(blocks.length); + List results = new ArrayList<>(blocks.length); do { results.add(createLocatedBlock(blocks[curBlk], curPos, mode)); curPos += blocks[curBlk].getNumBytes(); @@ -802,9 +805,9 @@ private List createLocatedBlockList( return results; } - private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks, + private LocatedBlock createLocatedBlock(final BlockInfo[] blocks, final long endPos, final AccessMode mode) throws IOException { - int curBlk = 0; + int curBlk; long curPos = 0; int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length; for (curBlk = 0; curBlk < nrBlocks; curBlk++) { @@ -817,8 +820,8 @@ private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks, return createLocatedBlock(blocks[curBlk], curPos, mode); } - - private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos, + + private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos, final AccessMode mode) throws IOException { final LocatedBlock lb = createLocatedBlock(blk, pos); if (mode != null) { @@ -828,8 +831,8 @@ private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final lon } /** @return a LocatedBlock for the given block */ - private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos - ) throws IOException { + private LocatedBlock createLocatedBlock(final BlockInfo blk, + final long pos) throws IOException { if (blk instanceof BlockInfoContiguousUnderConstruction) { if (blk.isComplete()) { throw new IOException( @@ -842,6 +845,7 @@ private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final lon final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); return newLocatedBlock(eb, storages, pos, false); } + // TODO support BlockInfoStripedUC // get block locations final int numCorruptNodes = countNodes(blk).corruptReplicas(); @@ -877,7 +881,7 @@ private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final lon } /** Create a LocatedBlocks. */ - public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks, + public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, final long fileSizeExcludeBlocksUnderConstruction, final boolean isFileUnderConstruction, final long offset, final long length, final boolean needBlockToken, @@ -900,7 +904,7 @@ public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks, final LocatedBlock lastlb; final boolean isComplete; if (!inSnapshot) { - final BlockInfoContiguous last = blocks[blocks.length - 1]; + final BlockInfo last = blocks[blocks.length - 1]; final long lastPos = last.isComplete()? fileSizeExcludeBlocksUnderConstruction - last.getNumBytes() : fileSizeExcludeBlocksUnderConstruction; @@ -1724,12 +1728,15 @@ private void processPendingReplications() { * reported by the datanode in the block report. */ static class StatefulBlockInfo { - final BlockInfoContiguousUnderConstruction storedBlock; + final BlockInfo storedBlock; // should be UC block final Block reportedBlock; final ReplicaState reportedState; - StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock, + StatefulBlockInfo(BlockInfo storedBlock, Block reportedBlock, ReplicaState reportedState) { + Preconditions.checkArgument( + storedBlock instanceof BlockInfoContiguousUnderConstruction || + storedBlock instanceof BlockInfoStripedUnderConstruction); this.storedBlock = storedBlock; this.reportedBlock = reportedBlock; this.reportedState = reportedState; @@ -2133,15 +2140,12 @@ private void processFirstBlockReport( // If block is under construction, add this replica to its list if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) { - ((BlockInfoContiguousUnderConstruction) storedBlock) - .addReplicaIfNotPresent(storageInfo, iblk, reportedState); + BlockInfo.addReplica(storedBlock, storageInfo, 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 - BlockInfoContiguousUnderConstruction blockUC = - (BlockInfoContiguousUnderConstruction) storedBlock; - if (namesystem.isInSnapshot(blockUC)) { - int numOfReplicas = blockUC.getNumExpectedLocations(); + if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) { + int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock); namesystem.incrementSafeBlockCount(numOfReplicas); } //and fall through to next clause @@ -2164,7 +2168,7 @@ private void reportDiff(DatanodeStorageInfo storageInfo, // place a delimiter in the list which separates blocks // that have been reported from those that have not Block delimiterBlock = new Block(); - BlockInfoContiguous delimiter = new BlockInfoContiguous(delimiterBlock, + BlockInfo delimiter = new BlockInfoContiguous(delimiterBlock, (short) 1); AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock); assert result == AddBlockResult.ADDED @@ -2294,9 +2298,8 @@ private BlockInfo processReportedBlock( } if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) { - toUC.add(new StatefulBlockInfo( - (BlockInfoContiguousUnderConstruction) storedBlock, - new Block(block), reportedState)); + toUC.add(new StatefulBlockInfo(storedBlock, new Block(block), + reportedState)); return storedBlock; } @@ -2487,9 +2490,8 @@ private boolean isBlockUnderConstruction(BlockInfo storedBlock, void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, DatanodeStorageInfo storageInfo) throws IOException { - BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock; - block.addReplicaIfNotPresent( - storageInfo, ucBlock.reportedBlock, ucBlock.reportedState); + BlockInfo block = ucBlock.storedBlock; + BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState); if (ucBlock.reportedState == ReplicaState.FINALIZED && (block.findStorageInfo(storageInfo) < 0)) { @@ -2549,7 +2551,8 @@ private Block addStoredBlock(final BlockInfo block, assert block != null && namesystem.hasWriteLock(); BlockInfo storedBlock; DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); - if (block instanceof BlockInfoContiguousUnderConstruction) { + if (block instanceof BlockInfoContiguousUnderConstruction || + block instanceof BlockInfoStripedUnderConstruction) { //refresh our copy in case the block got completed in another thread storedBlock = getStoredBlock(block); } else { @@ -2565,7 +2568,6 @@ private Block addStoredBlock(final BlockInfo block, return block; } BlockCollection bc = storedBlock.getBlockCollection(); - assert bc != null : "Block must belong to a file"; // add block to the datanode AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock); @@ -3493,8 +3495,8 @@ public void checkReplication(BlockCollection bc) { * replicated. */ public boolean checkBlocksProperlyReplicated( - String src, BlockInfoContiguous[] blocks) { - for (BlockInfoContiguous b: blocks) { + String src, BlockInfo[] blocks) { + for (BlockInfo b: blocks) { if (!b.isComplete()) { final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)b; @@ -3563,7 +3565,7 @@ boolean blockHasEnoughRacks(Block b) { if (!this.shouldCheckForEnoughRacks) { return true; } - boolean enoughRacks = false;; + boolean enoughRacks = false; Collection corruptNodes = corruptReplicas.getNodes(b); int numExpectedReplicas = getReplication(b); @@ -3609,21 +3611,15 @@ public long getMissingReplOneBlocksCount() { return this.neededReplications.getCorruptReplOneBlockSize(); } - public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block, + public BlockInfo addBlockCollection(BlockInfo block, BlockCollection bc) { - // TODO - return (BlockInfoContiguous) blocksMap.addBlockCollection(block, bc); + return blocksMap.addBlockCollection(block, bc); } public BlockCollection getBlockCollection(Block b) { return blocksMap.getBlockCollection(b); } - /** @return an iterator of the datanodes. */ - public Iterable getStorages(final Block block) { - return blocksMap.getStorages(block); - } - public int numCorruptReplicas(Block block) { return corruptReplicas.numCorruptReplicas(block); } @@ -3656,26 +3652,6 @@ private void removeFromExcessReplicateMap(Block block) { public int getCapacity() { return blocksMap.getCapacity(); } - - /** - * Return a range of corrupt replica block ids. Up to numExpectedBlocks - * blocks starting at the next block after startingBlockId are returned - * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId - * is null, up to numExpectedBlocks blocks are returned from the beginning. - * If startingBlockId cannot be found, null is returned. - * - * @param numExpectedBlocks Number of block ids to return. - * 0 <= numExpectedBlocks <= 100 - * @param startingBlockId Block id from which to start. If null, start at - * beginning. - * @return Up to numExpectedBlocks blocks from startingBlockId if it exists - * - */ - public long[] getCorruptReplicaBlockIds(int numExpectedBlocks, - Long startingBlockId) { - return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks, - startingBlockId); - } /** * Return an iterator over the set of blocks for which there are no replicas. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java index 79d77137f2904..a1290a0c63d6b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java @@ -369,7 +369,7 @@ private void rescanCacheDirectives() { * @param file The file. */ private void rescanFile(CacheDirective directive, INodeFile file) { - BlockInfoContiguous[] blockInfos = file.getBlocks(); + BlockInfo[] blockInfos = file.getBlocks(); // Increment the "needed" statistics directive.addFilesNeeded(1); @@ -394,7 +394,7 @@ private void rescanFile(CacheDirective directive, INodeFile file) { } long cachedTotal = 0; - for (BlockInfoContiguous blockInfo : blockInfos) { + for (BlockInfo blockInfo : blockInfos) { if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) { // We don't try to cache blocks that are under construction. LOG.trace("Directive {}: can't cache block {} because it is in state " @@ -452,8 +452,8 @@ private void rescanFile(CacheDirective directive, INodeFile file) { file.getFullPathName(), cachedTotal, neededTotal); } - private String findReasonForNotCaching(CachedBlock cblock, - BlockInfoContiguous blockInfo) { + private String findReasonForNotCaching(CachedBlock cblock, + BlockInfo blockInfo) { if (blockInfo == null) { // Somehow, a cache report with the block arrived, but the block // reports from the DataNode haven't (yet?) described such a block. @@ -513,7 +513,7 @@ private void rescanCachedBlockMap() { iter.remove(); } } - BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId())); + BlockInfo blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId())); String reason = findReasonForNotCaching(cblock, blockInfo); int neededCached = 0; if (reason != null) { @@ -627,7 +627,7 @@ private void addNewPendingCached(final int neededCached, List pendingCached) { // To figure out which replicas can be cached, we consult the // blocksMap. We don't want to try to cache a corrupt replica, though. - BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId())); + BlockInfo blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId())); if (blockInfo == null) { LOG.debug("Block {}: can't add new cached replicas," + " because there is no record of this block " + @@ -665,7 +665,7 @@ private void addNewPendingCached(final int neededCached, Iterator it = datanode.getPendingCached().iterator(); while (it.hasNext()) { CachedBlock cBlock = it.next(); - BlockInfoContiguous info = + BlockInfo info = namesystem.getStoredBlock(new Block(cBlock.getBlockId())); if (info != null) { pendingBytes -= info.getNumBytes(); @@ -675,7 +675,7 @@ private void addNewPendingCached(final int neededCached, // Add pending uncached blocks from effective capacity while (it.hasNext()) { CachedBlock cBlock = it.next(); - BlockInfoContiguous info = + BlockInfo info = namesystem.getStoredBlock(new Block(cBlock.getBlockId())); if (info != null) { pendingBytes += info.getNumBytes(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java index 3f22f51e879f1..de5b1f00af6a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java @@ -143,6 +143,7 @@ private static INodeFile[] verifySrcFiles(FSDirectory fsd, String[] srcs, throw new HadoopIllegalArgumentException("concat: source file " + src + " is invalid or empty or underConstruction"); } + // source file's preferred block size cannot be greater than the target // file if (srcINodeFile.getPreferredBlockSize() > @@ -152,6 +153,11 @@ private static INodeFile[] verifySrcFiles(FSDirectory fsd, String[] srcs, + " which is greater than the target file's preferred block size " + targetINode.getPreferredBlockSize()); } + // TODO currently we do not support concatenating EC files + if (srcINodeFile.isStriped()) { + throw new HadoopIllegalArgumentException("concat: the src file " + src + + " is with striped blocks"); + } si.add(srcINodeFile); } @@ -228,7 +234,7 @@ static void unprotectedConcat(FSDirectory fsd, INodesInPath targetIIP, int count = 0; for (INodeFile nodeToRemove : srcList) { if(nodeToRemove != null) { - nodeToRemove.setBlocks(null); + nodeToRemove.setContiguousBlocks(null); nodeToRemove.getParent().removeChild(nodeToRemove); fsd.getINodeMap().remove(nodeToRemove); count++; 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 8fdd2d7de55e8..1be2ad090f421 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 @@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; @@ -919,7 +920,7 @@ void unprotectedTruncate(String src, String clientName, String clientMachine, unprotectedTruncate(iip, newLength, collectedBlocks, mtime, null); if(! onBlockBoundary) { - BlockInfoContiguous oldBlock = file.getLastBlock(); + BlockInfo oldBlock = file.getLastBlock(); Block tBlk = getFSNamesystem().prepareFileForTruncate(iip, clientName, clientMachine, file.computeFileSize() - newLength, @@ -928,7 +929,7 @@ void unprotectedTruncate(String src, String clientName, String clientMachine, tBlk.getNumBytes() == truncateBlock.getNumBytes() : "Should be the same block."; if(oldBlock.getBlockId() != tBlk.getBlockId() && - !file.isBlockInLatestSnapshot(oldBlock)) { + !file.isBlockInLatestSnapshot((BlockInfoContiguous) oldBlock)) { getBlockManager().removeBlockFromMap(oldBlock); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java index 83e52bc05c711..5672ef31f72eb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java @@ -44,8 +44,8 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; @@ -773,10 +773,10 @@ public void logCloseFile(String path, INodeFile newNode) { public void logAddBlock(String path, INodeFile file) { Preconditions.checkArgument(file.isUnderConstruction()); - BlockInfoContiguous[] blocks = file.getBlocks(); + BlockInfo[] blocks = file.getBlocks(); Preconditions.checkState(blocks != null && blocks.length > 0); - BlockInfoContiguous pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null; - BlockInfoContiguous lastBlock = blocks[blocks.length - 1]; + BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null; + BlockInfo lastBlock = blocks[blocks.length - 1]; AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path) .setPenultimateBlock(pBlock).setLastBlock(lastBlock); logEdit(op); 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 476ff36cacad7..ce93ad7036b38 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 @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -506,7 +507,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, } INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path); // add the new block to the INodeFile - addNewBlock(fsDir, addBlockOp, oldFile); + addNewBlock(addBlockOp, oldFile); break; } case OP_SET_REPLICATION: { @@ -939,15 +940,15 @@ private static String formatEditLogReplayError(EditLogInputStream in, /** * Add a new block into the given INodeFile + * TODO support adding striped block */ - private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file) - throws IOException { - BlockInfoContiguous[] oldBlocks = file.getBlocks(); + private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException { + BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); Block newBlock= op.getLastBlock(); if (pBlock != null) { // the penultimate block is not null - Preconditions.checkState(oldBlocks != null && oldBlocks.length > 0); + assert oldBlocks != null && oldBlocks.length > 0; // compare pBlock with the last block of oldBlocks Block oldLastBlock = oldBlocks[oldBlocks.length - 1]; if (oldLastBlock.getBlockId() != pBlock.getBlockId() @@ -977,12 +978,13 @@ private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file) /** * Update in-memory data structures with new block information. + * TODO support adding striped block * @throws IOException */ 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 +993,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // 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; 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 ec2babd28f4dc..ff9baf4f42f56 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,6 +52,7 @@ 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -686,7 +687,7 @@ private void addToParent(INodeDirectory parent, INode child) { 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++) { @@ -958,9 +959,9 @@ LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) { 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); } 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 d966c696de857..b8dfa518b3e67 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 @@ -42,9 +42,13 @@ 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.protocol.proto.HdfsProtos.StripedBlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext; @@ -53,6 +57,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto; +import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.StripedBlocksFeature; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto; @@ -211,7 +216,7 @@ public static INodeDirectory loadINodeDirectory(INodeSection.INode n, 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)); @@ -347,16 +352,30 @@ private INodeFile loadINodeFile(INodeSection.INode n) { loadXAttrs(f.getXAttrs(), state.getStringTable()))); } + FileWithStripedBlocksFeature stripeFeature = null; + if (f.hasStripedBlocks()) { + StripedBlocksFeature sb = f.getStripedBlocks(); + stripeFeature = file.addStripedBlocksFeature(); + for (StripedBlockProto sp : sb.getBlocksList()) { + stripeFeature.addBlock(PBHelper.convert(sp)); + } + } + // under-construction information if (f.hasFileUC()) { INodeSection.FileUnderConstructionFeature uc = f.getFileUC(); file.toUnderConstruction(uc.getClientName(), uc.getClientMachine()); - if (blocks.length > 0) { - BlockInfoContiguous lastBlk = file.getLastBlock(); - // replace the last block of file - file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction( - lastBlk, replication)); + BlockInfo lastBlk = file.getLastBlock(); + // replace the last block of file + final BlockInfo ucBlk; + if (stripeFeature != null) { + BlockInfoStriped striped = (BlockInfoStriped) lastBlk; + ucBlk = new BlockInfoStripedUnderConstruction(striped, + striped.getDataBlockNum(), striped.getParityBlockNum()); + } else { + ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication); } + file.setBlock(file.numBlocks() - 1, ucBlk); } return file; } @@ -630,6 +649,19 @@ private void save(OutputStream out, INodeFile n) throws IOException { } } + FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature(); + if (sb != null) { + StripedBlocksFeature.Builder builder = + StripedBlocksFeature.newBuilder(); + BlockInfoStriped[] sblocks = sb.getBlocks(); + if (sblocks != null) { + for (BlockInfoStriped sblk : sblocks) { + builder.addBlocks(PBHelper.convert(sblk)); + } + } + b.setStripedBlocks(builder.build()); + } + FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature(); if (uc != null) { INodeSection.FileUnderConstructionFeature f = @@ -658,7 +690,7 @@ private void save(OutputStream out, INodeSymlink n) throws IOException { r.writeDelimitedTo(out); } - private final INodeSection.INode.Builder buildINodeCommon(INode n) { + private INodeSection.INode.Builder buildINodeCommon(INode n) { return INodeSection.INode.newBuilder() .setId(n.getId()) .setName(ByteString.copyFrom(n.getLocalNameBytes())); 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 d0313dbec30c3..6097c71dac081 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,8 +203,10 @@ 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; @@ -2009,6 +2011,11 @@ boolean truncateInternal(String src, long newLength, final BlockStoragePolicy lpPolicy = blockManager.getStoragePolicy("LAZY_PERSIST"); + // not support truncating file with striped blocks + if (file.isStriped()) { + throw new UnsupportedOperationException( + "Cannot truncate file with striped block " + src); + } if (lpPolicy != null && lpPolicy.getId() == file.getStoragePolicyID()) { throw new UnsupportedOperationException( @@ -2090,8 +2097,12 @@ Block prepareFileForTruncate(INodesInPath iip, leaseManager.addLease( file.getFileUnderConstructionFeature().getClientName(), file.getId()); boolean shouldRecoverNow = (newBlock == null); - BlockInfoContiguous oldBlock = file.getLastBlock(); - boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock); + + BlockInfo oldBlock = file.getLastBlock(); + assert oldBlock instanceof BlockInfoContiguous; + + boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, + (BlockInfoContiguous) oldBlock); if(newBlock == null) { newBlock = (shouldCopyOnTruncate) ? createNewBlock(file.isStriped()) : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(), @@ -2106,7 +2117,8 @@ Block prepareFileForTruncate(INodesInPath iip, file.getPreferredBlockReplication()); truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta); truncatedBlockUC.setTruncateBlock(oldBlock); - file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock)); + file.convertLastBlockToUC(truncatedBlockUC, + blockManager.getStorages(oldBlock)); getBlockManager().addBlockCollection(truncatedBlockUC, file); NameNode.stateChangeLog.info("BLOCK* prepareFileForTruncate: " @@ -2494,6 +2506,13 @@ private LocatedBlock appendFileInternal(FSPermissionChecker pc, + src + " for client " + clientMachine); } INodeFile myFile = INodeFile.valueOf(inode, src, true); + + // not support appending file with striped blocks + if (myFile.isStriped()) { + throw new UnsupportedOperationException( + "Cannot truncate file with striped block " + src); + } + final BlockStoragePolicy lpPolicy = blockManager.getStoragePolicy("LAZY_PERSIST"); if (lpPolicy != null && @@ -2505,7 +2524,8 @@ private LocatedBlock appendFileInternal(FSPermissionChecker pc, recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, src, holder, clientMachine, false); - final BlockInfoContiguous lastBlock = myFile.getLastBlock(); + final BlockInfoContiguous lastBlock = + (BlockInfoContiguous) myFile.getLastBlock(); // Check that the block has at least minimum replication. if(lastBlock != null && lastBlock.isComplete() && !getBlockManager().isSufficientlyReplicated(lastBlock)) { @@ -2561,7 +2581,7 @@ LocatedBlock prepareFileForAppend(String src, INodesInPath iip, } } } 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]); @@ -2740,7 +2760,7 @@ void recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip, 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( @@ -3066,13 +3086,13 @@ boolean checkFileProgress(String src, INodeFile v, boolean checkall) { .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 }); } } - + /** * Change the indicated filename. * @deprecated Use {@link #renameTo(String, String, boolean, @@ -3243,7 +3263,7 @@ void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) { for (Block b : blocks.getToDeleteList()) { if (trackBlockCounts) { - BlockInfoContiguous bi = getStoredBlock(b); + BlockInfo bi = getStoredBlock(b); if (bi.isComplete()) { numRemovedComplete++; if (bi.numNodes() >= blockManager.minReplication) { @@ -3467,10 +3487,10 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, 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()) @@ -3505,12 +3525,12 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, // 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 : + boolean penultimateBlockMinReplication = penultimateBlock == null || blockManager.checkMinReplication(penultimateBlock); switch(lastBlockState) { @@ -3540,6 +3560,7 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, throw new AlreadyBeingCreatedException(message); case UNDER_CONSTRUCTION: case UNDER_RECOVERY: + // TODO support Striped block's recovery final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock; // determine if last block was intended to be truncated @@ -3651,14 +3672,13 @@ void finalizeINodeFileUnderConstruction( blockManager.checkReplication(pendingFile); } - public BlockInfoContiguous getStoredBlock(Block block) { - return (BlockInfoContiguous) blockManager.getStoredBlock(block); + public BlockInfo getStoredBlock(Block block) { + return blockManager.getStoredBlock(block); } @Override - public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) { + public boolean isInSnapshot(BlockCollection bc) { assert hasReadLock(); - final BlockCollection bc = blockUC.getBlockCollection(); if (bc == null || !(bc instanceof INodeFile) || !bc.isUnderConstruction()) { return false; @@ -3703,7 +3723,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, waitForLoadingFSImage(); writeLock(); boolean copyTruncate = false; - BlockInfoContiguousUnderConstruction truncatedBlock = null; + BlockInfo truncatedBlock = null; try { checkOperation(OperationCategory.WRITE); // If a DN tries to commit to the standby, the recovery will @@ -3711,7 +3731,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, checkNameNodeSafeMode( "Cannot commitBlockSynchronization while in safe mode"); - final BlockInfoContiguous storedBlock = getStoredBlock( + final BlockInfo storedBlock = getStoredBlock( ExtendedBlock.getLocalBlock(oldBlock)); if (storedBlock == null) { if (deleteblock) { @@ -3760,9 +3780,8 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, return; } - truncatedBlock = (BlockInfoContiguousUnderConstruction) iFile - .getLastBlock(); - long recoveryId = truncatedBlock.getBlockRecoveryId(); + truncatedBlock = iFile.getLastBlock(); + long recoveryId = BlockInfo.getBlockRecoveryId(truncatedBlock); copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId(); if(recoveryId != newgenerationstamp) { throw new IOException("The recovery id " + newgenerationstamp @@ -3776,8 +3795,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, if (remove) { blockManager.removeBlock(storedBlock); } - } - else { + } else { // update last block if(!copyTruncate) { storedBlock.setGenerationStamp(newgenerationstamp); @@ -3825,9 +3843,9 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]), trimmedStorages.toArray(new String[trimmedStorages.size()])); if(copyTruncate) { - iFile.setLastBlock(truncatedBlock, trimmedStorageInfos); + iFile.convertLastBlockToUC(truncatedBlock, trimmedStorageInfos); } else { - iFile.setLastBlock(storedBlock, trimmedStorageInfos); + iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos); if (closeFile) { blockManager.markBlockReplicasAsCorrupt(storedBlock, oldGenerationStamp, oldNumBytes, trimmedStorageInfos); @@ -3838,7 +3856,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, if (closeFile) { if(copyTruncate) { src = closeFileCommitBlocks(iFile, truncatedBlock); - if(!iFile.isBlockInLatestSnapshot(storedBlock)) { + if(!iFile.isBlockInLatestSnapshot((BlockInfoContiguous) storedBlock)) { blockManager.removeBlock(storedBlock); } } else { @@ -3872,7 +3890,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, * @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(); @@ -4163,7 +4181,7 @@ private void clearCorruptLazyPersistFiles() while (it.hasNext()) { Block b = it.next(); - BlockInfoContiguous blockInfo = getStoredBlock(b); + BlockInfo blockInfo = getStoredBlock(b); if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) { filesToDelete.add(blockInfo.getBlockCollection()); } @@ -5105,7 +5123,7 @@ public void decrementSafeBlockCount(Block b) { 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()); } @@ -5667,7 +5685,7 @@ private INodeFile checkUCBlock(ExtendedBlock block, + "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 + @@ -5796,8 +5814,11 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, assert hasWriteLock(); // check the vadility of the block and lease holder name final INodeFile pendingFile = checkUCBlock(oldBlock, clientName); - final BlockInfoContiguousUnderConstruction blockinfo - = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock(); + final BlockInfo lastBlock = pendingFile.getLastBlock(); + // when updating pipeline, the last block must be contiguous block + assert lastBlock instanceof BlockInfoContiguousUnderConstruction; + BlockInfoContiguousUnderConstruction blockinfo = + (BlockInfoContiguousUnderConstruction) lastBlock; // check new GS & length: this is not expected if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() || 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 1ebdde645d707..a8e2e00e5105c 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,8 +20,10 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; /** @@ -58,12 +60,12 @@ public String getClientMachine() { */ 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) + assert !lastBlock.isComplete() : "The last block for path " + f.getFullPathName() - + " is not a BlockInfoUnderConstruction when updating its length"; + + " is not a BlockInfoUnderConstruction when updating its length"; lastBlock.setNumBytes(lastBlockLength); } @@ -74,11 +76,10 @@ void updateLengthOfLastBlock(INodeFile f, long lastBlockLength) */ 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 = - (BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1]; + && !blocks[blocks.length - 1].isComplete()) { + BlockInfo lastUC = blocks[blocks.length - 1]; if (lastUC.getNumBytes() == 0) { // this is a 0-sized block. do not need check its UC state here collectedBlocks.addDeleteBlock(lastUC); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java new file mode 100644 index 0000000000000..47445be5b6b03 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; + +/** + * Feature for file with striped blocks + */ +class FileWithStripedBlocksFeature implements INode.Feature { + private BlockInfoStriped[] blocks; + + FileWithStripedBlocksFeature() { + blocks = new BlockInfoStriped[0]; + } + + FileWithStripedBlocksFeature(BlockInfoStriped[] blocks) { + Preconditions.checkArgument(blocks != null); + this.blocks = blocks; + } + + BlockInfoStriped[] getBlocks() { + return this.blocks; + } + + void setBlock(int index, BlockInfoStriped blk) { + blocks[index] = blk; + } + + BlockInfoStriped getLastBlock() { + return blocks == null || blocks.length == 0 ? + null : blocks[blocks.length - 1]; + } + + int numBlocks() { + return blocks == null ? 0 : blocks.length; + } + + void updateBlockCollection(INodeFile file) { + if (blocks != null) { + for (BlockInfoStriped blk : blocks) { + blk.setBlockCollection(file); + } + } + } + + private void setBlocks(BlockInfoStriped[] blocks) { + this.blocks = blocks; + } + + void addBlock(BlockInfoStriped newBlock) { + if (this.blocks == null) { + this.setBlocks(new BlockInfoStriped[]{newBlock}); + } else { + int size = this.blocks.length; + BlockInfoStriped[] newlist = new BlockInfoStriped[size + 1]; + System.arraycopy(this.blocks, 0, newlist, 0, size); + newlist[size] = newBlock; + this.setBlocks(newlist); + } + } + + boolean removeLastBlock(Block oldblock) { + if (blocks == null || blocks.length == 0) { + return false; + } + int newSize = blocks.length - 1; + if (!blocks[newSize].equals(oldblock)) { + return false; + } + + //copy to a new list + BlockInfoStriped[] newlist = new BlockInfoStriped[newSize]; + System.arraycopy(blocks, 0, newlist, 0, newSize); + setBlocks(newlist); + return true; + } + + void truncateStripedBlocks(int n) { + final BlockInfoStriped[] newBlocks; + if (n == 0) { + newBlocks = new BlockInfoStriped[0]; + } else { + newBlocks = new BlockInfoStriped[n]; + System.arraycopy(getBlocks(), 0, newBlocks, 0, n); + } + // set new blocks + setBlocks(newBlocks); + } + + void clear() { + this.blocks = null; + } +} 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 dec8a3b798d26..ac58812b7ccc4 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.namenode; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION; import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID; import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID; @@ -37,12 +38,12 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature; @@ -174,6 +175,31 @@ && getAclFeature() == other.getAclFeature() && getXAttrFeature() == other.getXAttrFeature(); } + /* Start of StripedBlock Feature */ + + public final FileWithStripedBlocksFeature getStripedBlocksFeature() { + return getFeature(FileWithStripedBlocksFeature.class); + } + + public FileWithStripedBlocksFeature addStripedBlocksFeature() { + assert blocks == null || blocks.length == 0: + "The file contains contiguous blocks"; + assert !isWithStripedBlocks(); + this.setFileReplication((short) 0); + FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature(); + addFeature(sb); + return sb; + } + + public boolean isWithStripedBlocks() { + return getStripedBlocksFeature() != null; + } + + /** Used to make sure there is no contiguous block related info */ + private boolean hasNoContiguousBlock() { + return (blocks == null || blocks.length == 0) && getFileReplication() == 0; + } + /* Start of Under-Construction Feature */ /** @@ -208,7 +234,7 @@ public INodeFile toCompleteFile(long mtime) { "file is no longer under construction"); FileUnderConstructionFeature uc = getFileUnderConstructionFeature(); if (uc != null) { - assertAllBlocksComplete(); + assertAllBlocksComplete(getBlocks()); removeFeature(uc); this.setModificationTime(mtime); } @@ -216,37 +242,56 @@ public INodeFile toCompleteFile(long mtime) { } /** Assert all blocks are complete. */ - private void assertAllBlocksComplete() { - if (blocks == null) { + private void assertAllBlocksComplete(BlockInfo[] blks) { + if (blks == null) { return; } - for (int i = 0; i < blocks.length; i++) { - Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize" + for (int i = 0; i < blks.length; i++) { + Preconditions.checkState(blks[i].isComplete(), "Failed to finalize" + " %s %s since blocks[%s] is non-complete, where blocks=%s.", - getClass().getSimpleName(), this, i, Arrays.asList(blocks)); + getClass().getSimpleName(), this, i, Arrays.asList(blks)); } } + /** + * Instead of adding a new block, this function is usually used while loading + * fsimage or converting the last block to UC/Complete. + */ @Override // BlockCollection - public void setBlock(int index, BlockInfoContiguous blk) { - this.blocks[index] = blk; + public void setBlock(int index, BlockInfo blk) { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + assert blk instanceof BlockInfoContiguous; + this.blocks[index] = (BlockInfoContiguous) blk; + } else { + assert blk instanceof BlockInfoStriped; + assert hasNoContiguousBlock(); + sb.setBlock(index, (BlockInfoStriped) blk); + } } @Override // BlockCollection, the file should be under construction - public BlockInfoContiguousUnderConstruction setLastBlock( - BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations) - throws IOException { + public void convertLastBlockToUC(BlockInfo lastBlock, + DatanodeStorageInfo[] locations) throws IOException { Preconditions.checkState(isUnderConstruction(), "file is no longer under construction"); - if (numBlocks() == 0) { throw new IOException("Failed to set last block: File is empty."); } - BlockInfoContiguousUnderConstruction ucBlock = - lastBlock.convertToBlockUnderConstruction( - BlockUCState.UNDER_CONSTRUCTION, locations); + + final BlockInfo ucBlock; + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + assert lastBlock instanceof BlockInfoContiguous; + ucBlock = ((BlockInfoContiguous) lastBlock) + .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); + } else { + assert hasNoContiguousBlock(); + assert lastBlock instanceof BlockInfoStriped; + ucBlock = ((BlockInfoStriped) lastBlock) + .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); + } setBlock(numBlocks() - 1, ucBlock); - return ucBlock; } /** @@ -256,21 +301,27 @@ public BlockInfoContiguousUnderConstruction setLastBlock( BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) { Preconditions.checkState(isUnderConstruction(), "file is no longer under construction"); - if (blocks == null || blocks.length == 0) { - return null; - } - int size_1 = blocks.length - 1; - if (!blocks[size_1].equals(oldblock)) { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + if (blocks == null || blocks.length == 0) { + return null; + } + int size_1 = blocks.length - 1; + if (!blocks[size_1].equals(oldblock)) { + return null; + } + + BlockInfoContiguousUnderConstruction uc = + (BlockInfoContiguousUnderConstruction)blocks[size_1]; + //copy to a new list + BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1]; + System.arraycopy(blocks, 0, newlist, 0, size_1); + setContiguousBlocks(newlist); + return uc; + } else { + assert hasNoContiguousBlock(); return null; } - - BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction)blocks[size_1]; - //copy to a new list - BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1]; - System.arraycopy(blocks, 0, newlist, 0, size_1); - setBlocks(newlist); - return uc; } /* End of Under-Construction Feature */ @@ -371,13 +422,15 @@ public short getPreferredBlockReplication() { } /** Set the replication factor of this file. */ - public final void setFileReplication(short replication) { + private void setFileReplication(short replication) { header = HeaderFormat.REPLICATION.BITS.combine(replication, header); } /** Set the replication factor of this file. */ public final INodeFile setFileReplication(short replication, int latestSnapshotId) throws QuotaExceededException { + Preconditions.checkState(!isWithStripedBlocks(), + "Cannot set replication to a file with striped blocks"); recordModification(latestSnapshotId); setFileReplication(replication); return this; @@ -415,37 +468,57 @@ public final void setStoragePolicyID(byte storagePolicyId, setStoragePolicyID(storagePolicyId); } - @Override + @Override // INodeFileAttributes public long getHeaderLong() { return header; } /** @return the blocks of the file. */ - @Override - public BlockInfoContiguous[] getBlocks() { + @Override // BlockCollection + public BlockInfo[] getBlocks() { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb != null) { + assert hasNoContiguousBlock(); + return sb.getBlocks(); + } else { + return this.blocks; + } + } + + /** Used by snapshot diff */ + public BlockInfoContiguous[] getContiguousBlocks() { return this.blocks; } /** @return blocks of the file corresponding to the snapshot. */ - public BlockInfoContiguous[] getBlocks(int snapshot) { - if(snapshot == CURRENT_STATE_ID || getDiffs() == null) + public BlockInfo[] getBlocks(int snapshot) { + if (snapshot == CURRENT_STATE_ID || getDiffs() == null) { return getBlocks(); + } + // find blocks stored in snapshot diffs (for truncate) FileDiff diff = getDiffs().getDiffById(snapshot); - BlockInfoContiguous[] snapshotBlocks = - diff == null ? getBlocks() : diff.getBlocks(); - if(snapshotBlocks != null) + // note that currently FileDiff can only store contiguous blocks + BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks(); + if (snapshotBlocks != null) { return snapshotBlocks; + } // Blocks are not in the current snapshot // Find next snapshot with blocks present or return current file blocks snapshotBlocks = getDiffs().findLaterSnapshotBlocks(snapshot); return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks; } - void updateBlockCollection() { - if (blocks != null) { + /** Used during concat to update the BlockCollection for each block */ + private void updateBlockCollection() { + if (blocks != null && blocks.length > 0) { for(BlockInfoContiguous b : blocks) { b.setBlockCollection(this); } + } else { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb != null) { + sb.updateBlockCollection(this); + } } } @@ -468,27 +541,27 @@ void concatBlocks(INodeFile[] inodes) { size += in.blocks.length; } - setBlocks(newlist); + setContiguousBlocks(newlist); updateBlockCollection(); } /** - * add a block to the block list + * add a contiguous block to the block list */ void addBlock(BlockInfoContiguous newblock) { if (this.blocks == null) { - this.setBlocks(new BlockInfoContiguous[]{newblock}); + this.setContiguousBlocks(new BlockInfoContiguous[]{newblock}); } else { int size = this.blocks.length; BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1]; System.arraycopy(this.blocks, 0, newlist, 0, size); newlist[size] = newblock; - this.setBlocks(newlist); + this.setContiguousBlocks(newlist); } } /** Set the blocks. */ - public void setBlocks(BlockInfoContiguous[] blocks) { + public void setContiguousBlocks(BlockInfoContiguous[] blocks) { this.blocks = blocks; } @@ -539,13 +612,19 @@ public void destroyAndCollectBlocks(ReclaimContext reclaimContext) { } public void clearFile(ReclaimContext reclaimContext) { - if (blocks != null && reclaimContext.collectedBlocks != null) { - for (BlockInfoContiguous blk : blocks) { + BlockInfo[] blks = getBlocks(); + if (blks != null && reclaimContext.collectedBlocks != null) { + for (BlockInfo blk : blks) { reclaimContext.collectedBlocks.addDeleteBlock(blk); blk.setBlockCollection(null); } } - setBlocks(null); + setContiguousBlocks(null); + + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb != null) { + sb.clear(); + } if (getAclFeature() != null) { AclStorage.removeAclFeature(getAclFeature()); } @@ -712,13 +791,27 @@ public final long computeFileSize(boolean includesLastUcBlock, */ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build(); - final Iterable blocks; + if (isStriped()) { + return storagespaceConsumedWithStriped(bsp); + } else { + return storagespaceConsumedWithReplication(bsp); + } + } + + public final QuotaCounts storagespaceConsumedWithStriped( + BlockStoragePolicy bsp) { + return null; + } + + public final QuotaCounts storagespaceConsumedWithReplication( + BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build(); + 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(); @@ -730,7 +823,7 @@ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { } final short replication = getPreferredBlockReplication(); - for (BlockInfoContiguous b : blocks) { + for (BlockInfo b : blocks) { long blockSize = b.isComplete() ? b.getNumBytes() : getPreferredBlockSize(); counts.addStorageSpace(blockSize * replication); @@ -746,24 +839,44 @@ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { return counts; } + public final short getReplication(int lastSnapshotId) { + if (lastSnapshotId != CURRENT_STATE_ID) { + return getFileReplication(lastSnapshotId); + } else { + return getBlockReplication(); + } + } + /** * Return the penultimate allocated block for this file. */ - BlockInfoContiguous getPenultimateBlock() { - if (blocks == null || blocks.length <= 1) { - return null; - } - return blocks[blocks.length - 2]; + BlockInfo getPenultimateBlock() { + BlockInfo[] blks = getBlocks(); + return (blks == null || blks.length <= 1) ? + null : blks[blks.length - 2]; } @Override - public BlockInfoContiguous getLastBlock() { - return blocks == null || blocks.length == 0? null: blocks[blocks.length-1]; + public BlockInfo getLastBlock() { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + return blocks == null || blocks.length == 0 ? + null : blocks[blocks.length - 1]; + } else { + assert hasNoContiguousBlock(); + return sb.getLastBlock(); + } } @Override public int numBlocks() { - return blocks == null ? 0 : blocks.length; + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + return blocks == null ? 0 : blocks.length; + } else { + assert hasNoContiguousBlock(); + return sb.numBlocks(); + } } @VisibleForTesting @@ -775,6 +888,7 @@ public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix, // only compare the first block out.print(", blocks="); out.print(blocks == null || blocks.length == 0? null: blocks[0]); + // TODO print striped blocks out.println(); } @@ -784,9 +898,10 @@ public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix, */ public long collectBlocksBeyondMax(final long max, final BlocksMapUpdateInfo collectedBlocks) { - final BlockInfoContiguous[] oldBlocks = getBlocks(); - if (oldBlocks == null) + final BlockInfo[] oldBlocks = getBlocks(); + if (oldBlocks == null) { return 0; + } // find the minimum n such that the size of the first n blocks > max int n = 0; long size = 0; @@ -865,21 +980,36 @@ void computeQuotaDeltaForTruncate( } void truncateBlocksTo(int n) { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + truncateContiguousBlocks(n); + } else { + sb.truncateStripedBlocks(n); + } + } + + private void truncateContiguousBlocks(int n) { final BlockInfoContiguous[] newBlocks; if (n == 0) { newBlocks = BlockInfoContiguous.EMPTY_ARRAY; } else { newBlocks = new BlockInfoContiguous[n]; - System.arraycopy(getBlocks(), 0, newBlocks, 0, n); + System.arraycopy(blocks, 0, newBlocks, 0, n); } // set new blocks - setBlocks(newBlocks); + setContiguousBlocks(newBlocks); } + /** + * This function is only called when block list is stored in snapshot + * diffs. Note that this can only happen when truncation happens with + * snapshots. Since we do not support truncation with striped blocks, + * we only need to handle contiguous blocks here. + */ public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks, BlocksMapUpdateInfo collectedBlocks) { - BlockInfoContiguous[] oldBlocks = getBlocks(); - if(snapshotBlocks == null || oldBlocks == null) + BlockInfoContiguous[] oldBlocks = this.blocks; + if (snapshotBlocks == null || oldBlocks == null) return; // Skip blocks in common between the file and the snapshot int n = 0; @@ -887,7 +1017,7 @@ public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks, oldBlocks[n] == snapshotBlocks[n]) { n++; } - truncateBlocksTo(n); + truncateContiguousBlocks(n); // Collect the remaining blocks of the file while(n < oldBlocks.length) { collectedBlocks.addDeleteBlock(oldBlocks[n++]); 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 0806f82974715..2c6de55e0aa04 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 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,15 +108,15 @@ synchronized long getNumUnderConstructionBlocks() { 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++; + } } - } LOG.info("Number of blocks under construction: " + numUCBlocks); return numUCBlocks; } 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 c535bd9258774..acf302e9a636c 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,6 +64,7 @@ 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; @@ -243,8 +244,7 @@ public void blockIdCK(String blockId) { //get blockInfo Block block = new Block(Block.getBlockId(blockId)); //find which file this block belongs to - BlockInfoContiguous blockInfo = namenode.getNamesystem() - .getStoredBlock(block); + BlockInfo blockInfo = namenode.getNamesystem().getStoredBlock(block); if(blockInfo == null) { out.println("Block "+ blockId +" " + NONEXISTENT_STATUS); LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java index 3442e7be6b7cb..4695c3f7dbf19 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java @@ -19,6 +19,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.util.RwLock; @@ -45,5 +46,5 @@ public interface Namesystem extends RwLock, SafeMode { public void checkOperation(OperationCategory read) throws StandbyException; - public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC); + public boolean isInSnapshot(BlockCollection bc); } \ No newline at end of file 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 87b370a9a7d43..74baec5f2c0de 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 @@ -239,15 +239,16 @@ private void loadFileDiffList(InputStream in, INodeFile file, int size) FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, pbf.getFileSize()); List bpl = pbf.getBlocksList(); - // TODO: also persist striped blocks + // in file diff there can only be contiguous blocks BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()]; for(int j = 0, e = bpl.size(); j < e; ++j) { Block blk = PBHelper.convert(bpl.get(j)); BlockInfoContiguous storedBlock = (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk); if(storedBlock == null) { - storedBlock = fsn.getBlockManager().addBlockCollection( - new BlockInfoContiguous(blk, copy.getFileReplication()), file); + storedBlock = (BlockInfoContiguous) fsn.getBlockManager() + .addBlockCollection(new BlockInfoContiguous(blk, + copy.getFileReplication()), file); } blocks[j] = storedBlock; } 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 62aaccd42407c..b722aa9f07cba 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 @@ -55,7 +55,9 @@ public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile, final FileDiff diff = super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy); if (withBlocks) { // Store blocks if this is the first update - diff.setBlocks(iNodeFile.getBlocks()); + BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks(); + assert blks != null; + diff.setBlocks(blks); } } @@ -118,7 +120,7 @@ void combineAndCollectSnapshotBlocks( (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks()); // Find later snapshot (or file itself) with blocks BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId()); - laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks; + laterBlocks = (laterBlocks == null) ? file.getContiguousBlocks() : laterBlocks; // Skip blocks, which belong to either the earlier or the later lists int i = 0; for(; i < removedBlocks.length; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto index 3bd1d9160d415..b6fd03396faf4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto @@ -91,6 +91,10 @@ message INodeSection { optional string clientMachine = 2; } + message StripedBlocksFeature { + repeated StripedBlockProto blocks = 1; + } + message AclFeatureProto { /** * An ACL entry is represented by a 32-bit integer in Big Endian @@ -139,6 +143,7 @@ message INodeSection { optional AclFeatureProto acl = 8; optional XAttrFeatureProto xAttrs = 9; optional uint32 storagePolicyID = 10; + optional StripedBlocksFeature stripedBlocks = 11; } message QuotaByStorageTypeEntryProto { 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 86fb46297d974..d281a64538a9c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -490,6 +490,16 @@ message BlockProto { optional uint64 numBytes = 3 [default = 0]; } +/** + * Striped block information. Besides the basic information for a block, + * it also contains the number of data/parity blocks. + */ +message StripedBlockProto { + required BlockProto block = 1; + optional uint32 dataBlockNum = 2; + optional uint32 parityBlockNum = 3; +} + /** * Block and datanodes where is it located */ 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 207d1bba8a3be..c6d44d19d3082 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,6 +108,7 @@ 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -1609,7 +1610,7 @@ public static void createKey(String keyName, MiniDFSCluster cluster, public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn, ExtendedBlock blk) { FSNamesystem fsn = nn.getNamesystem(); - BlockInfoContiguous storedBlock = fsn.getStoredBlock(blk.getLocalBlock()); + BlockInfo storedBlock = fsn.getStoredBlock(blk.getLocalBlock()); assertTrue("Block " + blk + " should be under construction, " + "got: " + storedBlock, storedBlock instanceof BlockInfoContiguousUnderConstruction); 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 2834aadbe0dc5..54e7108ccc5c1 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 @@ -1246,8 +1246,8 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() (DatanodeStorageInfo.AddBlockResult.ADDED); ucBlock.addStorage(storage, ucBlock); - when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any())) - .thenReturn(ucBlock); + BlockInfo lastBlk = mbc.getLastBlock(); + when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock); bm.convertLastBlockToUnderConstruction(mbc, 0L); 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 a417c3d79d6dc..301ee25ad6b85 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.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 void testAddBlock() throws Exception { // 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 void testAddBlock() throws Exception { // 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 void testAddBlockUC() throws Exception { 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()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java index 06dfade3f4f0c..a2ef7b2d18f89 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.junit.After; import org.junit.Before; @@ -75,7 +76,7 @@ public void testAddBlockGroup() throws Exception { final Path file1 = new Path("/file1"); DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L); INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile(); - BlockInfoContiguous[] file1Blocks = file1Node.getBlocks(); + BlockInfo[] file1Blocks = file1Node.getBlocks(); assertEquals(2, file1Blocks.length); assertEquals(GROUP_SIZE, file1Blocks[0].numNodes()); assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP, 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 1fbe160a4711f..f372bec3ba62e 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.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 @@ private void verifyFileBlocks(String file, " 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]; 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 5653df5a47d8e..8a87ed3bbccce 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 @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.EnumSet; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.junit.Assert; import org.apache.hadoop.conf.Configuration; @@ -39,7 +40,6 @@ 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.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 @@ private void testPersistHelper(Configuration conf) throws IOException { 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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java index 222f22b3faca6..ffa1451d02472 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; 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.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; @@ -1035,7 +1036,8 @@ public void testTruncateRecovery() throws IOException { iip = fsn.getFSDirectory().getINodesInPath(src, true); file = iip.getLastINode().asFile(); file.recordModification(iip.getLatestSnapshotId(), true); - assertThat(file.isBlockInLatestSnapshot(file.getLastBlock()), is(true)); + assertThat(file.isBlockInLatestSnapshot( + (BlockInfoContiguous) file.getLastBlock()), is(true)); initialGenStamp = file.getLastBlock().getGenerationStamp(); // Test that prepareFileForTruncate sets up copy-on-write truncate fsn.writeLock(); 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 7f31f84bc288b..eabd0c8c938e3 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.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 void testFsckError() throws Exception { // 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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java index 85072d1fc0fa1..7bffb3399f17c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 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.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; @@ -108,14 +108,14 @@ static INodeFile assertBlockCollection(String path, int numBlocks, final FSDirectory dir, final BlockManager blkManager) throws Exception { final INodeFile file = INodeFile.valueOf(dir.getINode(path), path); assertEquals(numBlocks, file.getBlocks().length); - for(BlockInfoContiguous b : file.getBlocks()) { + for(BlockInfo b : file.getBlocks()) { assertBlockCollection(blkManager, file, b); } return file; } static void assertBlockCollection(final BlockManager blkManager, - final INodeFile file, final BlockInfoContiguous b) { + final INodeFile file, final BlockInfo b) { Assert.assertSame(b, blkManager.getStoredBlock(b)); Assert.assertSame(file, blkManager.getBlockCollection(b)); Assert.assertSame(file, b.getBlockCollection()); @@ -146,10 +146,10 @@ public void testDeletionWithSnapshots() throws Exception { { final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir, blockmanager); - BlockInfoContiguous[] blocks = f2.getBlocks(); + BlockInfo[] blocks = f2.getBlocks(); hdfs.delete(sub2, true); // The INode should have been removed from the blocksMap - for(BlockInfoContiguous b : blocks) { + for(BlockInfo b : blocks) { assertNull(blockmanager.getBlockCollection(b)); } } @@ -177,7 +177,7 @@ public void testDeletionWithSnapshots() throws Exception { // Check the block information for file0 final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir, blockmanager); - BlockInfoContiguous[] blocks0 = f0.getBlocks(); + BlockInfo[] blocks0 = f0.getBlocks(); // Also check the block information for snapshot of file0 Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0", @@ -187,7 +187,7 @@ public void testDeletionWithSnapshots() throws Exception { // Delete file0 hdfs.delete(file0, true); // Make sure the blocks of file0 is still in blocksMap - for(BlockInfoContiguous b : blocks0) { + for(BlockInfo b : blocks0) { assertNotNull(blockmanager.getBlockCollection(b)); } assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager); @@ -201,7 +201,7 @@ public void testDeletionWithSnapshots() throws Exception { hdfs.deleteSnapshot(sub1, "s1"); // Make sure the first block of file0 is still in blocksMap - for(BlockInfoContiguous b : blocks0) { + for(BlockInfo b : blocks0) { assertNotNull(blockmanager.getBlockCollection(b)); } assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager); @@ -293,7 +293,7 @@ public void testDeletionWithZeroSizeBlock() throws Exception { hdfs.append(bar); INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile(); - BlockInfoContiguous[] blks = barNode.getBlocks(); + BlockInfo[] blks = barNode.getBlocks(); assertEquals(1, blks.length); assertEquals(BLOCKSIZE, blks[0].getNumBytes()); ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]); @@ -331,7 +331,7 @@ public void testDeletionWithZeroSizeBlock2() throws Exception { hdfs.append(bar); INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile(); - BlockInfoContiguous[] blks = barNode.getBlocks(); + BlockInfo[] blks = barNode.getBlocks(); assertEquals(1, blks.length); ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]); cluster.getNameNodeRpc() @@ -370,7 +370,7 @@ public void testDeletionWithZeroSizeBlock3() throws Exception { hdfs.append(bar); INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile(); - BlockInfoContiguous[] blks = barNode.getBlocks(); + BlockInfo[] blks = barNode.getBlocks(); assertEquals(1, blks.length); ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]); cluster.getNameNodeRpc() @@ -421,7 +421,7 @@ public void testDeletionOfLaterBlocksWithZeroSizeFirstBlock() throws Exception { out.write(testData); out.close(); INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile(); - BlockInfoContiguous[] blks = barNode.getBlocks(); + BlockInfo[] blks = barNode.getBlocks(); assertEquals(1, blks.length); assertEquals(testData.length, blks[0].getNumBytes()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java index cdd655ed5fa34..139a37e1d7d77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology; 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.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; @@ -262,12 +262,12 @@ public void testDeleteCurrentFileDirectory() throws Exception { DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed); final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection( tempFile.toString(), 1, fsdir, blockmanager); - BlockInfoContiguous[] blocks = temp.getBlocks(); + BlockInfo[] blocks = temp.getBlocks(); hdfs.delete(tempDir, true); // check dir's quota usage checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3); // check blocks of tempFile - for (BlockInfoContiguous b : blocks) { + for (BlockInfo b : blocks) { assertNull(blockmanager.getBlockCollection(b)); } @@ -344,7 +344,7 @@ public void testDeleteCurrentFileDirectory() throws Exception { // while deletion, we add diff for subsub and metaChangeFile1, and remove // newFile checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4); - for (BlockInfoContiguous b : blocks) { + for (BlockInfo b : blocks) { assertNull(blockmanager.getBlockCollection(b)); } @@ -481,7 +481,7 @@ public void testDeleteEarliestSnapshot2() throws Exception { final INodeFile toDeleteFileNode = TestSnapshotBlocksMap .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager); - BlockInfoContiguous[] blocks = toDeleteFileNode.getBlocks(); + BlockInfo[] blocks = toDeleteFileNode.getBlocks(); // create snapshot s0 on dir SnapshotTestHelper.createSnapshot(hdfs, dir, "s0"); @@ -507,7 +507,7 @@ public void testDeleteEarliestSnapshot2() throws Exception { // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and // metaChangeFile's replication factor decreases checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE); - for (BlockInfoContiguous b : blocks) { + for (BlockInfo b : blocks) { assertNull(blockmanager.getBlockCollection(b)); } @@ -801,7 +801,7 @@ private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr, FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1); INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection( file14_s2.toString(), 1, fsdir, blockmanager); - BlockInfoContiguous[] blocks_14 = file14Node.getBlocks(); + BlockInfo[] blocks_14 = file14Node.getBlocks(); TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir, blockmanager); @@ -838,7 +838,7 @@ private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr, modDirStr + "file15"); assertFalse(hdfs.exists(file14_s1)); assertFalse(hdfs.exists(file15_s1)); - for (BlockInfoContiguous b : blocks_14) { + for (BlockInfo b : blocks_14) { assertNull(blockmanager.getBlockCollection(b)); } From 9af1f4779b646fb2f09b5e36447c8b8abe920a7c Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 27 Apr 2015 11:08:16 -0700 Subject: [PATCH 011/212] Addendum fix for HDFS-7749 to be compatible with HDFS-7993 --- .../hadoop/hdfs/server/namenode/NamenodeFsck.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 acf302e9a636c..e47e22e509a8a 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 @@ -634,9 +634,9 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res report.append(" Live_repl=" + liveReplicas); if (showLocations || showRacks || showReplicaDetails) { StringBuilder sb = new StringBuilder("["); - Iterable storages = bm.getStorages(block.getLocalBlock()); - for (Iterator iterator = storages.iterator(); iterator.hasNext();) { - DatanodeStorageInfo storage = iterator.next(); + DatanodeStorageInfo[] storages = bm.getStorages(storedBlock); + for (int i = 0; i < storages.length; i++) { + DatanodeStorageInfo storage = storages[i]; DatanodeDescriptor dnDesc = storage.getDatanodeDescriptor(); if (showRacks) { sb.append(NodeBase.getPath(dnDesc)); @@ -645,7 +645,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res .getStorageType())); } if (showReplicaDetails) { - LightWeightLinkedSet blocksExcess = + LightWeightLinkedSet blocksExcess = bm.excessReplicateMap.get(dnDesc.getDatanodeUuid()); Collection corruptReplicas = bm.getCorruptReplicas(block.getLocalBlock()); @@ -666,7 +666,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res sb.append("LIVE)"); } } - if (iterator.hasNext()) { + if (i < storages.length - 1) { sb.append(", "); } } From 1e1e93040748231dc913190aec1e031c379d8271 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 2 Mar 2015 13:44:33 -0800 Subject: [PATCH 012/212] HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao. --- .../blockmanagement/BlockIdManager.java | 31 +++- .../server/blockmanagement/BlockInfo.java | 4 +- .../blockmanagement/BlockInfoContiguous.java | 7 +- .../blockmanagement/BlockInfoStriped.java | 8 +- .../server/blockmanagement/BlockManager.java | 44 ++++-- .../server/blockmanagement/BlocksMap.java | 20 ++- .../blockmanagement/DecommissionManager.java | 9 +- .../server/namenode/FSDirWriteFileOp.java | 20 +-- .../hdfs/server/namenode/FSEditLogLoader.java | 69 ++++++--- .../hdfs/server/namenode/FSImageFormat.java | 12 +- .../server/namenode/FSImageFormatPBINode.java | 5 +- .../namenode/FSImageFormatProtobuf.java | 9 +- .../hdfs/server/namenode/FSNamesystem.java | 18 +-- .../hdfs/server/namenode/INodeFile.java | 25 ++- .../namenode/NameNodeLayoutVersion.java | 3 +- .../hadoop-hdfs/src/main/proto/fsimage.proto | 1 + .../server/namenode/TestAddBlockgroup.java | 85 ---------- .../server/namenode/TestAddStripedBlocks.java | 146 ++++++++++++++++++ 18 files changed, 340 insertions(+), 176 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index fa800c572f25c..8a71f18715a8c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -104,20 +104,37 @@ SequentialBlockIdGenerator getBlockIdGenerator() { } /** - * Sets the maximum allocated block ID for this filesystem. This is + * Sets the maximum allocated contiguous block ID for this filesystem. This is * the basis for allocating new block IDs. */ - public void setLastAllocatedBlockId(long blockId) { + public void setLastAllocatedContiguousBlockId(long blockId) { blockIdGenerator.skipTo(blockId); } /** - * Gets the maximum sequentially allocated block ID for this filesystem + * Gets the maximum sequentially allocated contiguous block ID for this + * filesystem */ - public long getLastAllocatedBlockId() { + public long getLastAllocatedContiguousBlockId() { return blockIdGenerator.getCurrentValue(); } + /** + * Sets the maximum allocated striped block ID for this filesystem. This is + * the basis for allocating new block IDs. + */ + public void setLastAllocatedStripedBlockId(long blockId) { + blockGroupIdGenerator.skipTo(blockId); + } + + /** + * Gets the maximum sequentially allocated striped block ID for this + * filesystem + */ + public long getLastAllocatedStripedBlockId() { + return blockGroupIdGenerator.getCurrentValue(); + } + /** * Sets the current generation stamp for legacy blocks */ @@ -189,11 +206,11 @@ public boolean isLegacyBlock(Block block) { /** * Increments, logs and then returns the block ID */ - public long nextBlockId() { + public long nextContiguousBlockId() { return blockIdGenerator.nextValue(); } - public long nextBlockGroupId() { + public long nextStripedBlockId() { return blockGroupIdGenerator.nextValue(); } @@ -217,7 +234,7 @@ public static boolean isStripedBlockID(long id) { return id < 0; } - public static long convertToGroupID(long id) { + public static long convertToStripedID(long id) { return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK); } 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 bf8e64f4ed7e1..8b7192521aaa2 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 @@ -169,6 +169,8 @@ public int getCapacity() { */ abstract void replaceBlock(BlockInfo newBlock); + public abstract boolean isStriped(); + /** * Find specified DatanodeDescriptor. * @return index or -1 if not found. @@ -336,7 +338,7 @@ public void setNext(LightWeightGSet.LinkedElement next) { } static BlockInfo copyOf(BlockInfo b) { - if (b instanceof BlockInfoContiguous) { + if (!b.isStriped()) { return new BlockInfoContiguous((BlockInfoContiguous) b); } else { return new BlockInfoStriped((BlockInfoStriped) b); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index e30e022eaa54e..d3051a300ffb9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -148,7 +148,7 @@ private int ensureCapacity(int num) { * happen only when replication is manually increased by the user. */ Object[] old = triplets; triplets = new Object[(last+num)*3]; - System.arraycopy(old, 0, triplets, 0, last*3); + System.arraycopy(old, 0, triplets, 0, last * 3); return last; } @@ -232,4 +232,9 @@ public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction( ucBlock.setBlockCollection(getBlockCollection()); return ucBlock; } + + @Override + public final boolean isStriped() { + return false; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 57de772f11f9b..8b458df82dceb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; /** @@ -57,7 +56,7 @@ public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) { this.setBlockCollection(b.getBlockCollection()); } - private short getTotalBlockNum() { + short getTotalBlockNum() { return (short) (dataBlockNum + parityBlockNum); } @@ -174,6 +173,11 @@ void replaceBlock(BlockInfo newBlock) { } } + @Override + public final boolean isStriped() { + return true; + } + @Override public int numNodes() { assert this.triplets != null : "BlockInfo is not initialized"; 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 80394aaf1cf7a..70e697e7e4516 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 @@ -582,11 +582,22 @@ public int getMaxReplicationStreams() { return maxReplicationStreams; } - /** - * @return true if the block has minimum replicas - */ - public boolean checkMinReplication(Block block) { - return (countNodes(block).liveReplicas() >= minReplication); + public int getDefaultStorageNum(BlockInfo block) { + return block.isStriped() ? + ((BlockInfoStriped) block).getTotalBlockNum() : defaultReplication; + } + + public short getMinStorageNum(BlockInfo block) { + return block.isStriped() ? + ((BlockInfoStriped) block).getDataBlockNum() : minReplication; + } + + public boolean checkMinStorage(BlockInfo block) { + return countNodes(block).liveReplicas() >= getMinStorageNum(block); + } + + public boolean checkMinStorage(BlockInfo block, int liveNum) { + return liveNum >= getMinStorageNum(block); } /** @@ -630,7 +641,7 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, return false; // already completed (e.g. by syncBlock) final boolean b = commitBlock(lastBlock, commitBlock); - if (countNodes(lastBlock).liveReplicas() >= minReplication) { + if (checkMinStorage(lastBlock)) { completeBlock(bc, bc.numBlocks() - 1, false); } return b; @@ -654,7 +665,7 @@ private BlockInfo completeBlock(final BlockCollection bc, } int numNodes = curBlock.numNodes(); - if (!force && numNodes < minReplication) { + if (!force && !checkMinStorage(curBlock, numNodes)) { throw new IOException("Cannot complete block: " + "block does not satisfy minimal replication requirement."); } @@ -698,9 +709,8 @@ private BlockInfo completeBlock(final BlockCollection bc, * when tailing edit logs as a Standby. */ public BlockInfo forceCompleteBlock(final BlockCollection bc, - final BlockInfoContiguousUnderConstruction block) throws IOException { - // TODO: support BlockInfoStripedUC for editlog - block.commitBlock(block); + final BlockInfo block) throws IOException { + BlockInfo.commitBlock(block, block); return completeBlock(bc, block, true); } @@ -751,7 +761,7 @@ public LocatedBlock convertLastBlockToUnderConstruction( // count in safe-mode. namesystem.adjustSafeModeBlockTotals( // decrement safe if we had enough - targets.length >= minReplication ? -1 : 0, + checkMinStorage(oldBlock, targets.length) ? -1 : 0, // always decrement total blocks -1); @@ -1197,8 +1207,8 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, NumberReplicas numberOfReplicas = countNodes(b.stored); boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= expectedReplicas; - boolean minReplicationSatisfied = - numberOfReplicas.liveReplicas() >= minReplication; + boolean minReplicationSatisfied = checkMinStorage(b.stored, + numberOfReplicas.liveReplicas()); boolean hasMoreCorruptReplicas = minReplicationSatisfied && (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) > expectedReplicas; @@ -2526,7 +2536,7 @@ private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported, // Now check for completion of blocks and safe block count int numCurrentReplica = countLiveNodes(storedBlock); if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED - && numCurrentReplica >= minReplication) { + && checkMinStorage(storedBlock, numCurrentReplica)) { completeBlock(storedBlock.getBlockCollection(), storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block @@ -2601,7 +2611,7 @@ private Block addStoredBlock(final BlockInfo block, + pendingReplications.getNumReplicas(storedBlock); if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED && - numLiveReplicas >= minReplication) { + checkMinStorage(storedBlock, numLiveReplicas)) { storedBlock = completeBlock(bc, storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block @@ -3283,6 +3293,8 @@ public void processIncrementalBlockReport(final DatanodeID nodeID, /** * Return the number of nodes hosting a given block, grouped * by the state of those replicas. + * For a striped block, this includes nodes storing blocks belonging to the + * striped block group. */ public NumberReplicas countNodes(Block b) { int decommissioned = 0; @@ -3438,7 +3450,7 @@ public BlockInfo getStoredBlock(Block block) { BlockInfo info = null; if (BlockIdManager.isStripedBlockID(block.getBlockId())) { info = blocksMap.getStoredBlock( - new Block(BlockIdManager.convertToGroupID(block.getBlockId()))); + new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); } if (info == null) { info = blocksMap.getStoredBlock(block); 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 59ff030b89d90..b093e5b143db4 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 @@ -43,8 +43,15 @@ private static class StorageIterator implements Iterator { @Override public boolean hasNext() { - return blockInfo != null && nextIdx < blockInfo.getCapacity() - && blockInfo.getDatanode(nextIdx) != null; + if (blockInfo == null) { + return false; + } + while (nextIdx < blockInfo.getCapacity() && + blockInfo.getDatanode(nextIdx) == null) { + // note that for striped blocks there may be null in the triplets + nextIdx++; + } + return nextIdx < blockInfo.getCapacity(); } @Override @@ -123,10 +130,13 @@ void removeBlock(Block block) { return; blockInfo.setBlockCollection(null); - // TODO: fix this logic for block group - for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) { + final int size = blockInfo instanceof BlockInfoContiguous ? + blockInfo.numNodes() : blockInfo.getCapacity(); + for(int idx = size - 1; idx >= 0; idx--) { DatanodeDescriptor dn = blockInfo.getDatanode(idx); - dn.removeBlock(blockInfo); // remove from the list and wipe the location + if (dn != null) { + dn.removeBlock(blockInfo); // remove from the list and wipe the location + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java index 5f7366e5a0481..8a3657611c123 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java @@ -544,7 +544,7 @@ private void processBlocksForDecomInternal( int underReplicatedInOpenFiles = 0; while (it.hasNext()) { numBlocksChecked++; - final BlockInfoContiguous block = it.next(); + final BlockInfo block = it.next(); // Remove the block from the list if it's no longer in the block map, // e.g. the containing file has been deleted if (blockManager.blocksMap.getStoredBlock(block) == null) { @@ -578,8 +578,9 @@ private void processBlocksForDecomInternal( } // Even if the block is under-replicated, - // it doesn't block decommission if it's sufficiently replicated - if (isSufficientlyReplicated(block, bc, num)) { + // it doesn't block decommission if it's sufficiently replicated + BlockInfoContiguous blk = (BlockInfoContiguous) block; + if (isSufficientlyReplicated(blk, bc, num)) { if (pruneSufficientlyReplicated) { it.remove(); } @@ -588,7 +589,7 @@ private void processBlocksForDecomInternal( // We've found an insufficiently replicated block. if (insufficientlyReplicated != null) { - insufficientlyReplicated.add(block); + insufficientlyReplicated.add(blk); } // Log if this is our first time through if (firstReplicationLog) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 307bd5942264a..325d9d34e8115 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -471,21 +471,20 @@ static INodeFile addFileForEditLog( assert fsd.hasWriteLock(); if (underConstruction) { newNode = newINodeFile(id, permissions, modificationTime, - modificationTime, replication, - preferredBlockSize, - storagePolicyId); + modificationTime, replication, preferredBlockSize, storagePolicyId); newNode.toUnderConstruction(clientName, clientMachine); } else { - newNode = newINodeFile(id, permissions, modificationTime, - atime, replication, - preferredBlockSize, - storagePolicyId); + newNode = newINodeFile(id, permissions, modificationTime, atime, + replication, preferredBlockSize, storagePolicyId); } newNode.setLocalName(localName); try { INodesInPath iip = fsd.addINode(existing, newNode); if (iip != null) { + if (newNode.isStriped()) { + newNode.addStripedBlocksFeature(); + } if (aclEntries != null) { AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID); } @@ -553,7 +552,7 @@ private static INodesInPath addFile( long modTime = now(); INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions, - modTime, modTime, replication, preferredBlockSize); + modTime, modTime, replication, preferredBlockSize); newNode.setLocalName(localName.getBytes(Charsets.UTF_8)); newNode.toUnderConstruction(clientName, clientMachine); @@ -561,12 +560,15 @@ private static INodesInPath addFile( fsd.writeLock(); try { newiip = fsd.addINode(existing, newNode); + if (newiip != null && newNode.isStriped()) { + newNode.addStripedBlocksFeature(); + } } finally { fsd.writeUnlock(); } if (newiip == null) { NameNode.stateChangeLog.info("DIR* addFile: failed to add " + - existing.getPath() + "/" + localName); + existing.getPath() + "/" + localName); return null; } 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 ce93ad7036b38..c3ab2d5843bb9 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 @@ -38,7 +38,9 @@ import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -413,7 +415,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - updateBlocks(fsDir, addCloseOp, iip, newFile); + // TODO whether the file is striped should later be retrieved from iip + updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped()); break; } case OP_CLOSE: { @@ -433,7 +436,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - updateBlocks(fsDir, addCloseOp, iip, file); + // TODO whether the file is striped should later be retrieved from iip + updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped()); // Now close the file if (!file.isUnderConstruction() && @@ -491,7 +495,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures - updateBlocks(fsDir, updateOp, iip, oldFile); + // TODO whether the file is striped should later be retrieved from iip + updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped()); if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); @@ -507,7 +512,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, } INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path); // add the new block to the INodeFile - addNewBlock(addBlockOp, oldFile); + // TODO whether the file is striped should later be retrieved from iip + addNewBlock(addBlockOp, oldFile, oldFile.isStriped()); break; } case OP_SET_REPLICATION: { @@ -787,8 +793,15 @@ fsDir, renameReservedPathsOnUpgrade(timesOp.path, logVersion), } case OP_ALLOCATE_BLOCK_ID: { AllocateBlockIdOp allocateBlockIdOp = (AllocateBlockIdOp) op; - fsNamesys.getBlockIdManager().setLastAllocatedBlockId( - allocateBlockIdOp.blockId); + if (BlockIdManager.isStripedBlockID(allocateBlockIdOp.blockId)) { + // ALLOCATE_BLOCK_ID is added for sequential block id, thus if the id + // is negative, it must belong to striped blocks + fsNamesys.getBlockIdManager().setLastAllocatedStripedBlockId( + allocateBlockIdOp.blockId); + } else { + fsNamesys.getBlockIdManager().setLastAllocatedContiguousBlockId( + allocateBlockIdOp.blockId); + } break; } case OP_ROLLING_UPGRADE_START: { @@ -940,9 +953,9 @@ private static String formatEditLogReplayError(EditLogInputStream in, /** * Add a new block into the given INodeFile - * TODO support adding striped block */ - private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException { + private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped) + throws IOException { BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); Block newBlock= op.getLastBlock(); @@ -950,7 +963,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException { if (pBlock != null) { // the penultimate block is not null assert oldBlocks != null && oldBlocks.length > 0; // compare pBlock with the last block of oldBlocks - Block oldLastBlock = oldBlocks[oldBlocks.length - 1]; + BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1]; if (oldLastBlock.getBlockId() != pBlock.getBlockId() || oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) { throw new IOException( @@ -960,29 +973,33 @@ private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException { } oldLastBlock.setNumBytes(pBlock.getNumBytes()); - if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) { - fsNamesys.getBlockManager().forceCompleteBlock(file, - (BlockInfoContiguousUnderConstruction) oldLastBlock); + if (!oldLastBlock.isComplete()) { + fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock); fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock); } } else { // the penultimate block is null Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0); } // add the new block - BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction( - newBlock, file.getPreferredBlockReplication()); - fsNamesys.getBlockManager().addBlockCollection(newBI, file); - file.addBlock(newBI); + final BlockInfo newBlockInfo; + if (isStriped) { + newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, + HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); + } else { + newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, + file.getPreferredBlockReplication()); + } + fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file); + file.addBlock(newBlockInfo); fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock); } /** * Update in-memory data structures with new block information. - * TODO support adding striped block * @throws IOException */ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, - INodesInPath iip, INodeFile file) throws IOException { + INodesInPath iip, INodeFile file, boolean isStriped) throws IOException { // Update its block list BlockInfo[] oldBlocks = file.getBlocks(); Block[] newBlocks = op.getBlocks(); @@ -1011,11 +1028,10 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, oldBlock.getGenerationStamp() != newBlock.getGenerationStamp(); oldBlock.setGenerationStamp(newBlock.getGenerationStamp()); - if (oldBlock instanceof BlockInfoContiguousUnderConstruction && + if (!oldBlock.isComplete() && (!isLastBlock || op.shouldCompleteLastBlock())) { changeMade = true; - fsNamesys.getBlockManager().forceCompleteBlock(file, - (BlockInfoContiguousUnderConstruction) oldBlock); + fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock); } if (changeMade) { // The state or gen-stamp of the block has changed. So, we may be @@ -1045,13 +1061,18 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // We're adding blocks for (int i = oldBlocks.length; i < newBlocks.length; i++) { Block newBlock = newBlocks[i]; - BlockInfoContiguous newBI; + final 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 // until several blocks in? - newBI = new BlockInfoContiguousUnderConstruction( - newBlock, file.getPreferredBlockReplication()); + if (isStriped) { + newBI = new BlockInfoStripedUnderConstruction(newBlock, + HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); + } else { + newBI = new BlockInfoContiguousUnderConstruction(newBlock, + file.getPreferredBlockReplication()); + } } else { // OP_CLOSE should add finalized blocks. This code path // is only executed when loading edits written by prior 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 ff9baf4f42f56..aef0b2811b924 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 @@ -359,7 +359,14 @@ public void load(File curFile) throws IOException { // read the max sequential block ID. long maxSequentialBlockId = in.readLong(); - namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId); + namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId( + maxSequentialBlockId); + if (NameNodeLayoutVersion.supports( + NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)) { + final long maxStripedBlockId = in.readLong(); + namesystem.getBlockIdManager().setLastAllocatedStripedBlockId( + maxStripedBlockId); + } } else { long startingGenStamp = namesystem.getBlockIdManager() @@ -1269,7 +1276,8 @@ void save(File newFile, FSImageCompression compression) throws IOException { out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV1()); out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV2()); out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampAtblockIdSwitch()); - out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedBlockId()); + out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedContiguousBlockId()); + out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedStripedBlockId()); out.writeLong(context.getTxId()); out.writeLong(sourceNamesystem.dir.getLastInodeId()); 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 b8dfa518b3e67..a58e37ef6bcd9 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 @@ -643,8 +643,9 @@ private void save(OutputStream out, INodeFile n) throws IOException { INodeSection.INodeFile.Builder b = buildINodeFile(n, parent.getSaverContext()); - if (n.getBlocks() != null) { - for (Block block : n.getBlocks()) { + BlockInfoContiguous[] cBlks = n.getContiguousBlocks(); + if (cBlks != null) { + for (Block block : cBlks) { b.addBlocks(PBHelper.convert(block)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java index 24afcae4e0748..4b75c2c9b18c5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java @@ -296,7 +296,11 @@ private void loadNameSystemSection(InputStream in) throws IOException { blockIdManager.setGenerationStampV1(s.getGenstampV1()); blockIdManager.setGenerationStampV2(s.getGenstampV2()); blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit()); - blockIdManager.setLastAllocatedBlockId(s.getLastAllocatedBlockId()); + blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId()); + if (s.hasLastAllocatedStripedBlockId()) { + blockIdManager.setLastAllocatedStripedBlockId( + s.getLastAllocatedStripedBlockId()); + } imgTxId = s.getTransactionId(); if (s.hasRollingUpgradeStartTime() && fsn.getFSImage().hasRollbackFSImage()) { @@ -536,7 +540,8 @@ private void saveNameSystemSection(FileSummary.Builder summary) .setGenstampV1(blockIdManager.getGenerationStampV1()) .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit()) .setGenstampV2(blockIdManager.getGenerationStampV2()) - .setLastAllocatedBlockId(blockIdManager.getLastAllocatedBlockId()) + .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId()) + .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId()) .setTransactionId(context.getTxId()); // We use the non-locked version of getNamespaceInfo here since 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 6097c71dac081..fca848e27a68f 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 @@ -206,7 +206,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; @@ -2099,7 +2098,7 @@ Block prepareFileForTruncate(INodesInPath iip, boolean shouldRecoverNow = (newBlock == null); BlockInfo oldBlock = file.getLastBlock(); - assert oldBlock instanceof BlockInfoContiguous; + assert !oldBlock.isStriped(); boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, (BlockInfoContiguous) oldBlock); @@ -3266,7 +3265,7 @@ void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) { BlockInfo bi = getStoredBlock(b); if (bi.isComplete()) { numRemovedComplete++; - if (bi.numNodes() >= blockManager.minReplication) { + if (blockManager.checkMinStorage(bi, bi.numNodes())) { numRemovedSafe++; } } @@ -3495,7 +3494,7 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, curBlock = blocks[nrCompleteBlocks]; if(!curBlock.isComplete()) break; - assert blockManager.checkMinReplication(curBlock) : + assert blockManager.checkMinStorage(curBlock) : "A COMPLETE block is not minimally replicated in " + src; } @@ -3530,8 +3529,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, BlockInfo penultimateBlock = pendingFile.getPenultimateBlock(); // If penultimate block doesn't exist then its minReplication is met - boolean penultimateBlockMinReplication = penultimateBlock == null || - blockManager.checkMinReplication(penultimateBlock); + boolean penultimateBlockMinStorage = penultimateBlock == null || + blockManager.checkMinStorage(penultimateBlock); switch(lastBlockState) { case COMPLETE: @@ -3539,8 +3538,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, break; case COMMITTED: // Close file if committed blocks are minimally replicated - if(penultimateBlockMinReplication && - blockManager.checkMinReplication(lastBlock)) { + if(penultimateBlockMinStorage && + blockManager.checkMinStorage(lastBlock)) { finalizeINodeFileUnderConstruction(src, pendingFile, iip.getLatestSnapshotId()); NameNode.stateChangeLog.warn("BLOCK*" @@ -3640,6 +3639,7 @@ void commitOrCompleteLastBlock( } // Adjust disk space consumption if required + // TODO: support EC files final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes(); if (diff > 0) { try { @@ -5634,7 +5634,7 @@ private long nextBlockId(boolean isStriped) throws IOException { assert hasWriteLock(); checkNameNodeSafeMode("Cannot get next block ID"); final long blockId = isStriped ? - blockIdManager.nextBlockGroupId() : blockIdManager.nextBlockId(); + blockIdManager.nextStripedBlockId() : blockIdManager.nextContiguousBlockId(); getEditLog().logAllocateBlockId(blockId); // NB: callers sync the log return blockId; 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 ac58812b7ccc4..bc25f254fdc20 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 @@ -86,7 +86,7 @@ public static INodeFile valueOf(INode inode, String path, boolean acceptNull) */ static enum HeaderFormat { PREFERRED_BLOCK_SIZE(null, 48, 1), - REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1), + REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0), STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH, 0); @@ -261,10 +261,10 @@ private void assertAllBlocksComplete(BlockInfo[] blks) { public void setBlock(int index, BlockInfo blk) { FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); if (sb == null) { - assert blk instanceof BlockInfoContiguous; + assert !blk.isStriped(); this.blocks[index] = (BlockInfoContiguous) blk; } else { - assert blk instanceof BlockInfoStriped; + assert blk.isStriped(); assert hasNoContiguousBlock(); sb.setBlock(index, (BlockInfoStriped) blk); } @@ -282,12 +282,12 @@ public void convertLastBlockToUC(BlockInfo lastBlock, final BlockInfo ucBlock; FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); if (sb == null) { - assert lastBlock instanceof BlockInfoContiguous; + assert !lastBlock.isStriped(); ucBlock = ((BlockInfoContiguous) lastBlock) .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); } else { assert hasNoContiguousBlock(); - assert lastBlock instanceof BlockInfoStriped; + assert lastBlock.isStriped(); ucBlock = ((BlockInfoStriped) lastBlock) .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); } @@ -548,7 +548,7 @@ void concatBlocks(INodeFile[] inodes) { /** * add a contiguous block to the block list */ - void addBlock(BlockInfoContiguous newblock) { + private void addContiguousBlock(BlockInfoContiguous newblock) { if (this.blocks == null) { this.setContiguousBlocks(new BlockInfoContiguous[]{newblock}); } else { @@ -560,6 +560,19 @@ void addBlock(BlockInfoContiguous newblock) { } } + /** add a striped or contiguous block */ + void addBlock(BlockInfo newblock) { + FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); + if (sb == null) { + assert !newblock.isStriped(); + addContiguousBlock((BlockInfoContiguous) newblock); + } else { + assert newblock.isStriped(); + assert hasNoContiguousBlock(); + sb.addBlock((BlockInfoStriped) newblock); + } + } + /** Set the blocks. */ public void setContiguousBlocks(BlockInfoContiguous[] blocks) { this.blocks = blocks; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java index d235e2b4ccb35..f93218fccc159 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java @@ -72,7 +72,8 @@ public static enum Feature implements LayoutFeature { BLOCK_STORAGE_POLICY(-60, "Block Storage policy"), TRUNCATE(-61, "Truncate"), APPEND_NEW_BLOCK(-62, "Support appending to new block"), - QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types"); + QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types"), + ERASURE_CODING(-64, "Support erasure coding"); private final FeatureInfo info; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto index b6fd03396faf4..3f3a71ec64f88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto @@ -73,6 +73,7 @@ message NameSystemSection { optional uint64 lastAllocatedBlockId = 5; optional uint64 transactionId = 6; optional uint64 rollingUpgradeStartTime = 7; + optional uint64 lastAllocatedStripedBlockId = 8; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java deleted file mode 100644 index a2ef7b2d18f89..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hdfs.server.namenode; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.hdfs.DFSTestUtil; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; - -import static org.junit.Assert.assertEquals; - -public class TestAddBlockgroup { - - public static final Log LOG = LogFactory.getLog(TestAddBlockgroup.class); - - private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + - HdfsConstants.NUM_PARITY_BLOCKS; - private final short NUM_DATANODES = GROUP_SIZE; - - private static final int BLOCKSIZE = 1024; - private static final short REPLICATION = 3; - - private MiniDFSCluster cluster; - private Configuration conf; - - @Before - public void setup() throws IOException { - conf = new Configuration(); - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES) - .build(); - cluster.waitActive(); - cluster.getFileSystem().setStoragePolicy(new Path("/"), - HdfsConstants.EC_STORAGE_POLICY_NAME); - } - - @After - public void tearDown() { - if (cluster != null) { - cluster.shutdown(); - } - } - - @Test - public void testAddBlockGroup() throws Exception { - DistributedFileSystem fs = cluster.getFileSystem(); - FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); - - final Path file1 = new Path("/file1"); - DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L); - INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile(); - BlockInfo[] file1Blocks = file1Node.getBlocks(); - assertEquals(2, file1Blocks.length); - assertEquals(GROUP_SIZE, file1Blocks[0].numNodes()); - assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP, - file1Blocks[1].getBlockId() - file1Blocks[0].getBlockId()); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java new file mode 100644 index 0000000000000..7226f519bbdef --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +public class TestAddStripedBlocks { + private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; + + private MiniDFSCluster cluster; + private DistributedFileSystem dfs; + + @Before + public void setup() throws IOException { + cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()) + .numDataNodes(GROUP_SIZE).build(); + cluster.waitActive(); + dfs = cluster.getFileSystem(); + dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testAddStripedBlock() throws Exception { + final Path file = new Path("/file1"); + // create an empty file + FSDataOutputStream out = null; + try { + out = dfs.create(file, (short) 1); + + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + LocatedBlock newBlock = cluster.getNamesystem().getAdditionalBlock( + file.toString(), fileNode.getId(), dfs.getClient().getClientName(), + null, null, null); + assertEquals(GROUP_SIZE, newBlock.getLocations().length); + assertEquals(GROUP_SIZE, newBlock.getStorageIDs().length); + + BlockInfo[] blocks = fileNode.getBlocks(); + assertEquals(1, blocks.length); + Assert.assertTrue(blocks[0].isStriped()); + + checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true); + } finally { + IOUtils.cleanup(null, out); + } + + // restart NameNode to check editlog + cluster.restartNameNode(true); + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + BlockInfo[] blocks = fileNode.getBlocks(); + assertEquals(1, blocks.length); + Assert.assertTrue(blocks[0].isStriped()); + checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); + + // save namespace, restart namenode, and check + dfs = cluster.getFileSystem(); + dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER); + dfs.saveNamespace(); + dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE); + cluster.restartNameNode(true); + fsdir = cluster.getNamesystem().getFSDirectory(); + fileNode = fsdir.getINode4Write(file.toString()).asFile(); + blocks = fileNode.getBlocks(); + assertEquals(1, blocks.length); + Assert.assertTrue(blocks[0].isStriped()); + checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); + } + + private void checkStripedBlockUC(BlockInfoStriped block, + boolean checkReplica) { + assertEquals(0, block.numNodes()); + Assert.assertFalse(block.isComplete()); + Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum()); + Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, + block.getParityBlockNum()); + Assert.assertEquals(0, + block.getBlockId() & HdfsConstants.BLOCK_GROUP_INDEX_MASK); + + final BlockInfoStripedUnderConstruction blockUC = + (BlockInfoStripedUnderConstruction) block; + Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, + blockUC.getBlockUCState()); + if (checkReplica) { + Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations()); + DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations(); + for (DataNode dn : cluster.getDataNodes()) { + Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages)); + } + } + } + + private boolean includeDataNode(DatanodeID dn, DatanodeStorageInfo[] storages) { + for (DatanodeStorageInfo storage : storages) { + if (storage.getDatanodeDescriptor().equals(dn)) { + return true; + } + } + return false; + } +} From c0945a8971097d56a37e6d0a4085df3f0b9d0589 Mon Sep 17 00:00:00 2001 From: drankye Date: Thu, 5 Mar 2015 22:51:52 +0800 Subject: [PATCH 013/212] HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 4 + .../hadoop/io/erasurecode/ECSchema.java | 203 ++++++++++++++++++ .../hadoop/io/erasurecode/TestECSchema.java | 54 +++++ 3 files changed, 261 insertions(+) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 7bbacf7a4e7e9..ee42c8498e117 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -12,3 +12,7 @@ HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng ( Kai Zheng ) + HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng + ( Kai Zheng ) + + diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java new file mode 100644 index 0000000000000..8dc3f4516e0f6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -0,0 +1,203 @@ +/** + * 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.io.erasurecode; + +import java.util.Collections; +import java.util.Map; + +/** + * Erasure coding schema to housekeeper relevant information. + */ +public class ECSchema { + public static final String NUM_DATA_UNITS_KEY = "k"; + public static final String NUM_PARITY_UNITS_KEY = "m"; + public static final String CODEC_NAME_KEY = "codec"; + public static final String CHUNK_SIZE_KEY = "chunkSize"; + public static final int DEFAULT_CHUNK_SIZE = 64 * 1024; // 64K + + private String schemaName; + private String codecName; + private Map options; + private int numDataUnits; + private int numParityUnits; + private int chunkSize; + + /** + * Constructor with schema name and provided options. Note the options may + * contain additional information for the erasure codec to interpret further. + * @param schemaName schema name + * @param options schema options + */ + public ECSchema(String schemaName, Map options) { + assert (schemaName != null && ! schemaName.isEmpty()); + + this.schemaName = schemaName; + + if (options == null || options.isEmpty()) { + throw new IllegalArgumentException("No schema options are provided"); + } + + String codecName = options.get(CODEC_NAME_KEY); + if (codecName == null || codecName.isEmpty()) { + throw new IllegalArgumentException("No codec option is provided"); + } + + int dataUnits = 0, parityUnits = 0; + try { + if (options.containsKey(NUM_DATA_UNITS_KEY)) { + dataUnits = Integer.parseInt(options.get(NUM_DATA_UNITS_KEY)); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Option value " + + options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + " is found. It should be an integer"); + } + + try { + if (options.containsKey(NUM_PARITY_UNITS_KEY)) { + parityUnits = Integer.parseInt(options.get(NUM_PARITY_UNITS_KEY)); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Option value " + + options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + " is found. It should be an integer"); + } + + initWith(codecName, dataUnits, parityUnits, options); + } + + /** + * Constructor with key parameters provided. Note the options may contain + * additional information for the erasure codec to interpret further. + * @param schemaName + * @param codecName + * @param numDataUnits + * @param numParityUnits + * @param options + */ + public ECSchema(String schemaName, String codecName, + int numDataUnits, int numParityUnits, + Map options) { + assert (schemaName != null && ! schemaName.isEmpty()); + assert (codecName != null && ! codecName.isEmpty()); + + this.schemaName = schemaName; + initWith(codecName, numDataUnits, numParityUnits, options); + } + + private void initWith(String codecName, int numDataUnits, int numParityUnits, + Map options) { + this.codecName = codecName; + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + + this.options = options != null ? Collections.unmodifiableMap(options) : + Collections.EMPTY_MAP; + + this.chunkSize = DEFAULT_CHUNK_SIZE; + try { + if (options.containsKey(CHUNK_SIZE_KEY)) { + this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY)); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Option value " + + options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + " is found. It should be an integer"); + } + + boolean isFine = numDataUnits > 0 && numParityUnits > 0 && chunkSize > 0; + if (! isFine) { + throw new IllegalArgumentException("Bad codec options are found"); + } + } + + /** + * Get the schema name + * @return schema name + */ + public String getSchemaName() { + return schemaName; + } + + /** + * Get the codec name + * @return codec name + */ + public String getCodecName() { + return codecName; + } + + /** + * Get erasure coding options + * @return encoding options + */ + public Map getOptions() { + return options; + } + + /** + * Get required data units count in a coding group + * @return count of data units + */ + public int getNumDataUnits() { + return numDataUnits; + } + + /** + * Get required parity units count in a coding group + * @return count of parity units + */ + public int getNumParityUnits() { + return numParityUnits; + } + + /** + * Get chunk buffer size for the erasure encoding/decoding. + * @return chunk buffer size + */ + public int getChunkSize() { + return chunkSize; + } + + /** + * Make a meaningful string representation for log output. + * @return string representation + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ECSchema=["); + + sb.append("Name=" + schemaName + ","); + sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ","); + sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ","); + sb.append(CHUNK_SIZE_KEY + "=" + chunkSize + ","); + + for (String opt : options.keySet()) { + boolean skip = (opt.equals(NUM_DATA_UNITS_KEY) || + opt.equals(NUM_PARITY_UNITS_KEY) || + opt.equals(CHUNK_SIZE_KEY)); + if (! skip) { + sb.append(opt + "=" + options.get(opt) + ","); + } + } + + sb.append("]"); + + return sb.toString(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java new file mode 100644 index 0000000000000..4285ef0185fa1 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java @@ -0,0 +1,54 @@ +/** + * 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.io.erasurecode; + +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import java.util.HashMap; +import java.util.Map; + +public class TestECSchema { + + @Test + public void testGoodSchema() { + String schemaName = "goodSchema"; + int numDataUnits = 6; + int numParityUnits = 3; + int chunkSize = 64 * 1024 * 1024; + String codec = "rs"; + String extraOption = "extraOption"; + String extraOptionValue = "extraOptionValue"; + + Map options = new HashMap(); + options.put(ECSchema.NUM_DATA_UNITS_KEY, String.valueOf(numDataUnits)); + options.put(ECSchema.NUM_PARITY_UNITS_KEY, String.valueOf(numParityUnits)); + options.put(ECSchema.CODEC_NAME_KEY, codec); + options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(chunkSize)); + options.put(extraOption, extraOptionValue); + + ECSchema schema = new ECSchema(schemaName, options); + System.out.println(schema.toString()); + + assertEquals(schemaName, schema.getSchemaName()); + assertEquals(numDataUnits, schema.getNumDataUnits()); + assertEquals(numParityUnits, schema.getNumParityUnits()); + assertEquals(chunkSize, schema.getChunkSize()); + assertEquals(codec, schema.getCodecName()); + assertEquals(extraOptionValue, schema.getOptions().get(extraOption)); + } +} From 39a0a85fb77872911089aac3f7792ab48d9eca68 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 5 Mar 2015 16:44:38 -0800 Subject: [PATCH 014/212] HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print striped blocks. Contributed by Takuya Fukudome. --- .../org/apache/hadoop/hdfs/server/namenode/INodeFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 bc25f254fdc20..31562f28f0248 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 @@ -900,8 +900,8 @@ public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix, out.print(", fileSize=" + computeFileSize(snapshotId)); // only compare the first block out.print(", blocks="); - out.print(blocks == null || blocks.length == 0? null: blocks[0]); - // TODO print striped blocks + BlockInfo[] blks = getBlocks(); + out.print(blks == null || blks.length == 0? null: blks[0]); out.println(); } From 8f89d7489df0d5c8236a1929c93e3f5ab7149031 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 9 Mar 2015 12:32:26 +0530 Subject: [PATCH 015/212] HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Contributed by Kai Zheng ) --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 + .../apache/hadoop/io/erasurecode/ECBlock.java | 80 ++++++ .../hadoop/io/erasurecode/ECBlockGroup.java | 82 ++++++ .../coder/AbstractErasureCoder.java | 63 +++++ .../coder/AbstractErasureCodingStep.java | 59 ++++ .../coder/AbstractErasureDecoder.java | 152 ++++++++++ .../coder/AbstractErasureEncoder.java | 50 ++++ .../io/erasurecode/coder/ErasureCoder.java | 77 +++++ .../erasurecode/coder/ErasureCodingStep.java | 55 ++++ .../io/erasurecode/coder/ErasureDecoder.java | 41 +++ .../coder/ErasureDecodingStep.java | 52 ++++ .../io/erasurecode/coder/ErasureEncoder.java | 39 +++ .../coder/ErasureEncodingStep.java | 49 ++++ .../erasurecode/coder/XorErasureDecoder.java | 78 +++++ .../erasurecode/coder/XorErasureEncoder.java | 45 +++ .../erasurecode/rawcoder/RawErasureCoder.java | 2 +- .../coder/TestErasureCoderBase.java | 266 ++++++++++++++++++ .../io/erasurecode/coder/TestXorCoder.java | 50 ++++ 18 files changed, 1241 insertions(+), 1 deletion(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index ee42c8498e117..c17a1bd9037a0 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -15,4 +15,6 @@ HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng ( Kai Zheng ) + HADOOP-11646. Erasure Coder API for encoding and decoding of block group + ( Kai Zheng via vinayakumarb ) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java new file mode 100644 index 0000000000000..956954a76720d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java @@ -0,0 +1,80 @@ +/** + * 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.io.erasurecode; + +/** + * A wrapper of block level data source/output that {@link ECChunk}s can be + * extracted from. For HDFS, it can be an HDFS block (250MB). Note it only cares + * about erasure coding specific logic thus avoids coupling with any HDFS block + * details. We can have something like HdfsBlock extend it. + */ +public class ECBlock { + + private boolean isParity; + private boolean isErased; + + /** + * A default constructor. isParity and isErased are false by default. + */ + public ECBlock() { + this(false, false); + } + + /** + * A constructor specifying isParity and isErased. + * @param isParity + * @param isErased + */ + public ECBlock(boolean isParity, boolean isErased) { + this.isParity = isParity; + this.isErased = isErased; + } + + /** + * Set true if it's for a parity block. + * @param isParity + */ + public void setParity(boolean isParity) { + this.isParity = isParity; + } + + /** + * Set true if the block is missing. + * @param isMissing + */ + public void setErased(boolean isMissing) { + this.isErased = isMissing; + } + + /** + * + * @return true if it's parity block, otherwise false + */ + public boolean isParity() { + return isParity; + } + + /** + * + * @return true if it's missing or corrupt due to erasure, otherwise false + */ + public boolean isErased() { + return isErased; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java new file mode 100644 index 0000000000000..2c851a50dfbb5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java @@ -0,0 +1,82 @@ +/** + * 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.io.erasurecode; + +/** + * A group of blocks or {@link ECBlock} incurred in an erasure coding task. + */ +public class ECBlockGroup { + + private ECBlock[] dataBlocks; + private ECBlock[] parityBlocks; + + /** + * A constructor specifying data blocks and parity blocks. + * @param dataBlocks + * @param parityBlocks + */ + public ECBlockGroup(ECBlock[] dataBlocks, ECBlock[] parityBlocks) { + this.dataBlocks = dataBlocks; + this.parityBlocks = parityBlocks; + } + + /** + * Get data blocks + * @return data blocks + */ + public ECBlock[] getDataBlocks() { + return dataBlocks; + } + + /** + * Get parity blocks + * @return parity blocks + */ + public ECBlock[] getParityBlocks() { + return parityBlocks; + } + + /** + * Any erased data block? + * @return true if any erased data block, false otherwise + */ + public boolean anyErasedDataBlock() { + for (int i = 0; i < dataBlocks.length; ++i) { + if (dataBlocks[i].isErased()) { + return true; + } + } + + return false; + } + + /** + * Any erased parity block? + * @return true if any erased parity block, false otherwise + */ + public boolean anyErasedParityBlock() { + for (int i = 0; i < parityBlocks.length; ++i) { + if (parityBlocks[i].isErased()) { + return true; + } + } + + return false; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java new file mode 100644 index 0000000000000..f2cc041e0b678 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -0,0 +1,63 @@ +/** + * 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.io.erasurecode.coder; + +/** + * A common class of basic facilities to be shared by encoder and decoder + * + * It implements the {@link ErasureCoder} interface. + */ +public abstract class AbstractErasureCoder implements ErasureCoder { + + private int numDataUnits; + private int numParityUnits; + private int chunkSize; + + @Override + public void initialize(int numDataUnits, int numParityUnits, + int chunkSize) { + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + this.chunkSize = chunkSize; + } + + @Override + public int getNumDataUnits() { + return numDataUnits; + } + + @Override + public int getNumParityUnits() { + return numParityUnits; + } + + @Override + public int getChunkSize() { + return chunkSize; + } + + @Override + public boolean preferNativeBuffer() { + return false; + } + + @Override + public void release() { + // Nothing to do by default + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java new file mode 100644 index 0000000000000..c429d49a36c1f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; + +/** + * Abstract class for common facilities shared by {@link ErasureEncodingStep} + * and {@link ErasureDecodingStep}. + * + * It implements {@link ErasureEncodingStep}. + */ +public abstract class AbstractErasureCodingStep implements ErasureCodingStep { + + private ECBlock[] inputBlocks; + private ECBlock[] outputBlocks; + + /** + * Constructor given input blocks and output blocks. + * @param inputBlocks + * @param outputBlocks + */ + public AbstractErasureCodingStep(ECBlock[] inputBlocks, + ECBlock[] outputBlocks) { + this.inputBlocks = inputBlocks; + this.outputBlocks = outputBlocks; + } + + @Override + public ECBlock[] getInputBlocks() { + return inputBlocks; + } + + @Override + public ECBlock[] getOutputBlocks() { + return outputBlocks; + } + + @Override + public void finish() { + // NOOP by default + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java new file mode 100644 index 0000000000000..df71528e20deb --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java @@ -0,0 +1,152 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; + +/** + * An abstract erasure decoder that's to be inherited by new decoders. + * + * It implements the {@link ErasureDecoder} interface. + */ +public abstract class AbstractErasureDecoder extends AbstractErasureCoder + implements ErasureDecoder { + + @Override + public ErasureCodingStep decode(ECBlockGroup blockGroup) { + return performDecoding(blockGroup); + } + + /** + * Perform decoding against a block blockGroup. + * @param blockGroup + * @return decoding step for caller to do the real work + */ + protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup); + + /** + * We have all the data blocks and parity blocks as input blocks for + * recovering by default. It's codec specific + * @param blockGroup + * @return + */ + protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { + ECBlock[] inputBlocks = new ECBlock[getNumParityUnits() + + getNumDataUnits()]; + + System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks, 0, + getNumParityUnits()); + System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks, + getNumParityUnits(), getNumDataUnits()); + + return inputBlocks; + } + + /** + * Which blocks were erased ? We only care data blocks here. Sub-classes can + * override this behavior. + * @param blockGroup + * @return output blocks to recover + */ + protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { + ECBlock[] outputBlocks = new ECBlock[ + getNumErasedBlocks(blockGroup.getDataBlocks())]; + + int idx = 0; + for (int i = 0; i < getNumDataUnits(); i++) { + if (blockGroup.getDataBlocks()[i].isErased()) { + outputBlocks[idx++] = blockGroup.getDataBlocks()[i]; + } + } + + return outputBlocks; + } + + /** + * Get the number of erased blocks in the block group. + * @param blockGroup + * @return number of erased blocks + */ + protected int getNumErasedBlocks(ECBlockGroup blockGroup) { + int num = getNumErasedBlocks(blockGroup.getParityBlocks()); + num += getNumErasedBlocks(blockGroup.getDataBlocks()); + return num; + } + + /** + * Find out how many blocks are erased. + * @param inputBlocks all the input blocks + * @return number of erased blocks + */ + protected static int getNumErasedBlocks(ECBlock[] inputBlocks) { + int numErased = 0; + for (int i = 0; i < inputBlocks.length; i++) { + if (inputBlocks[i].isErased()) { + numErased ++; + } + } + + return numErased; + } + + /** + * Get indexes of erased blocks from inputBlocks + * @param inputBlocks + * @return indexes of erased blocks from inputBlocks + */ + protected int[] getErasedIndexes(ECBlock[] inputBlocks) { + int numErased = getNumErasedBlocks(inputBlocks); + if (numErased == 0) { + return new int[0]; + } + + int[] erasedIndexes = new int[numErased]; + int i = 0, j = 0; + for (; i < inputBlocks.length && j < erasedIndexes.length; i++) { + if (inputBlocks[i].isErased()) { + erasedIndexes[j++] = i; + } + } + + return erasedIndexes; + } + + /** + * Get erased input blocks from inputBlocks + * @param inputBlocks + * @return an array of erased blocks from inputBlocks + */ + protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) { + int numErased = getNumErasedBlocks(inputBlocks); + if (numErased == 0) { + return new ECBlock[0]; + } + + ECBlock[] erasedBlocks = new ECBlock[numErased]; + int i = 0, j = 0; + for (; i < inputBlocks.length && j < erasedBlocks.length; i++) { + if (inputBlocks[i].isErased()) { + erasedBlocks[j++] = inputBlocks[i]; + } + } + + return erasedBlocks; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java new file mode 100644 index 0000000000000..80b987513f9c3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java @@ -0,0 +1,50 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; + +/** + * An abstract erasure encoder that's to be inherited by new encoders. + * + * It implements the {@link ErasureEncoder} interface. + */ +public abstract class AbstractErasureEncoder extends AbstractErasureCoder + implements ErasureEncoder { + + @Override + public ErasureCodingStep encode(ECBlockGroup blockGroup) { + return performEncoding(blockGroup); + } + + /** + * Perform encoding against a block group. + * @param blockGroup + * @return encoding step for caller to do the real work + */ + protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup); + + protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { + return blockGroup.getDataBlocks(); + } + + protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { + return blockGroup.getParityBlocks(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java new file mode 100644 index 0000000000000..68875c06ad833 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java @@ -0,0 +1,77 @@ +/** + * 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.io.erasurecode.coder; + +/** + * An erasure coder to perform encoding or decoding given a group. Generally it + * involves calculating necessary internal steps according to codec logic. For + * each step,it calculates necessary input blocks to read chunks from and output + * parity blocks to write parity chunks into from the group. It also takes care + * of appropriate raw coder to use for the step. And encapsulates all the + * necessary info (input blocks, output blocks and raw coder) into a step + * represented by {@link ErasureCodingStep}. ErasureCoder callers can use the + * step to do the real work with retrieved input and output chunks. + * + * Note, currently only one coding step is supported. Will support complex cases + * of multiple coding steps. + * + */ +public interface ErasureCoder { + + /** + * Initialize with the important parameters for the code. + * @param numDataUnits how many data inputs for the coding + * @param numParityUnits how many parity outputs the coding generates + * @param chunkSize the size of the input/output buffer + */ + public void initialize(int numDataUnits, int numParityUnits, int chunkSize); + + /** + * The number of data input units for the coding. A unit can be a byte, + * chunk or buffer or even a block. + * @return count of data input units + */ + public int getNumDataUnits(); + + /** + * The number of parity output units for the coding. A unit can be a byte, + * chunk, buffer or even a block. + * @return count of parity output units + */ + public int getNumParityUnits(); + + /** + * Chunk buffer size for the input/output + * @return chunk buffer size + */ + public int getChunkSize(); + + /** + * Tell if native or off-heap buffer is preferred or not. It's for callers to + * decide how to allocate coding chunk buffers, either on heap or off heap. + * It will return false by default. + * @return true if native buffer is preferred for performance consideration, + * otherwise false. + */ + public boolean preferNativeBuffer(); + + /** + * Release the resources if any. Good chance to invoke RawErasureCoder#release. + */ + public void release(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java new file mode 100644 index 0000000000000..a3b177f2d2f46 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java @@ -0,0 +1,55 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECChunk; + +/** + * Erasure coding step that's involved in encoding/decoding of a block group. + */ +public interface ErasureCodingStep { + + /** + * Input blocks of readable data involved in this step, may be data blocks + * or parity blocks. + * @return input blocks + */ + public ECBlock[] getInputBlocks(); + + /** + * Output blocks of writable buffers involved in this step, may be data + * blocks or parity blocks. + * @return output blocks + */ + public ECBlock[] getOutputBlocks(); + + /** + * Perform encoding or decoding given the input chunks, and generated results + * will be written to the output chunks. + * @param inputChunks + * @param outputChunks + */ + public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks); + + /** + * Notify erasure coder that all the chunks of input blocks are processed so + * the coder can be able to update internal states, considering next step. + */ + public void finish(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java new file mode 100644 index 0000000000000..dfd9e545b502c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java @@ -0,0 +1,41 @@ +/** + * 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.io.erasurecode.coder; + + +import org.apache.hadoop.io.erasurecode.ECBlockGroup; + +/** + * Erasure decoder interface to perform decoding given a block group. + * + * It extends {@link ErasureCoder}. + */ +public interface ErasureDecoder extends ErasureCoder { + + /** + * Perform the decoding given a blockGroup. By default it will try the best to + * attempt to recover all the missing blocks according to the codec logic. + * + * Note, currently only one coding step is supported. Will support complex + * cases of multiple coding steps. + * + * @param blockGroup + */ + public ErasureCodingStep decode(ECBlockGroup blockGroup); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java new file mode 100644 index 0000000000000..980c5805de1c6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java @@ -0,0 +1,52 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECChunk; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; + +/** + * Erasure decoding step, a wrapper of all the necessary information to perform + * a decoding step involved in the whole process of decoding a block group. + */ +public class ErasureDecodingStep extends AbstractErasureCodingStep { + private int[] erasedIndexes; + private RawErasureDecoder rawDecoder; + + /** + * The constructor with all the necessary info. + * @param inputBlocks + * @param erasedIndexes the indexes of erased blocks in inputBlocks array + * @param outputBlocks + * @param rawDecoder + */ + public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes, + ECBlock[] outputBlocks, + RawErasureDecoder rawDecoder) { + super(inputBlocks, outputBlocks); + this.erasedIndexes = erasedIndexes; + this.rawDecoder = rawDecoder; + } + + @Override + public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) { + rawDecoder.decode(inputChunks, erasedIndexes, outputChunks); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java new file mode 100644 index 0000000000000..e837d229dcab0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java @@ -0,0 +1,39 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlockGroup; + +/** + * Erasure encoder interface to perform encoding given a block group. + * + * It extends {@link ErasureCoder}. + */ +public interface ErasureEncoder extends ErasureCoder { + + /** + * Calculate the encoding steps given a block blockGroup. + * + * Note, currently only one coding step is supported. Will support complex + * cases of multiple coding steps. + * + * @param blockGroup + */ + public ErasureCodingStep encode(ECBlockGroup blockGroup); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java new file mode 100644 index 0000000000000..bd7587fa3c2dc --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java @@ -0,0 +1,49 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECChunk; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; + +/** + * Erasure encoding step, a wrapper of all the necessary information to perform + * an encoding step involved in the whole process of encoding a block group. + */ +public class ErasureEncodingStep extends AbstractErasureCodingStep { + + private RawErasureEncoder rawEncoder; + + /** + * The constructor with all the necessary info. + * @param inputBlocks + * @param outputBlocks + * @param rawEncoder + */ + public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks, + RawErasureEncoder rawEncoder) { + super(inputBlocks, outputBlocks); + this.rawEncoder = rawEncoder; + } + + @Override + public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) { + rawEncoder.encode(inputChunks, outputChunks); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java new file mode 100644 index 0000000000000..9a6f6e2df9941 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java @@ -0,0 +1,78 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder; + +/** + * Xor erasure decoder that decodes a block group. + * + * It implements {@link ErasureDecoder}. + */ +public class XorErasureDecoder extends AbstractErasureDecoder { + + @Override + protected ErasureCodingStep performDecoding(final ECBlockGroup blockGroup) { + // May be configured + RawErasureDecoder rawDecoder = new XorRawDecoder(); + rawDecoder.initialize(getNumDataUnits(), + getNumParityUnits(), getChunkSize()); + + ECBlock[] inputBlocks = getInputBlocks(blockGroup); + + return new ErasureDecodingStep(inputBlocks, + getErasedIndexes(inputBlocks), + getOutputBlocks(blockGroup), rawDecoder); + } + + /** + * Which blocks were erased ? For XOR it's simple we only allow and return one + * erased block, either data or parity. + * @param blockGroup + * @return output blocks to recover + */ + @Override + protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { + /** + * If more than one blocks (either data or parity) erased, then it's not + * edible to recover. We don't have the check here since it will be done + * by upper level: ErasreCoder call can be avoid if not possible to recover + * at all. + */ + int erasedNum = getNumErasedBlocks(blockGroup); + ECBlock[] outputBlocks = new ECBlock[erasedNum]; + + int idx = 0; + for (int i = 0; i < getNumParityUnits(); i++) { + if (blockGroup.getParityBlocks()[i].isErased()) { + outputBlocks[idx++] = blockGroup.getParityBlocks()[i]; + } + } + + for (int i = 0; i < getNumDataUnits(); i++) { + if (blockGroup.getDataBlocks()[i].isErased()) { + outputBlocks[idx++] = blockGroup.getDataBlocks()[i]; + } + } + + return outputBlocks; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java new file mode 100644 index 0000000000000..db6e3db16e6f2 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java @@ -0,0 +1,45 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder; + +/** + * Xor erasure encoder that encodes a block group. + * + * It implements {@link ErasureEncoder}. + */ +public class XorErasureEncoder extends AbstractErasureEncoder { + + @Override + protected ErasureCodingStep performEncoding(final ECBlockGroup blockGroup) { + // May be configured + RawErasureEncoder rawEncoder = new XorRawEncoder(); + rawEncoder.initialize(getNumDataUnits(), + getNumParityUnits(), getChunkSize()); + + ECBlock[] inputBlocks = getInputBlocks(blockGroup); + + return new ErasureEncodingStep(inputBlocks, + getOutputBlocks(blockGroup), rawEncoder); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java index 6e07cf1c94583..91a9abfe2aad4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -71,7 +71,7 @@ public interface RawErasureCoder { public boolean preferNativeBuffer(); /** - * Should be called when release this blockcoder. Good chance to release encoding + * Should be called when release this coder. Good chance to release encoding * or decoding buffers */ public void release(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java new file mode 100644 index 0000000000000..ca5c1c9c5356d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -0,0 +1,266 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECChunk; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.TestCoderBase; + +/** + * Erasure coder test base with utilities. + */ +public abstract class TestErasureCoderBase extends TestCoderBase { + protected Class encoderClass; + protected Class decoderClass; + + protected int numChunksInBlock = 16; + + /** + * It's just a block for this test purpose. We don't use HDFS block here + * at all for simple. + */ + protected static class TestBlock extends ECBlock { + private ECChunk[] chunks; + + // For simple, just assume the block have the chunks already ready. + // In practice we need to read/write chunks from/to the block via file IO. + public TestBlock(ECChunk[] chunks) { + this.chunks = chunks; + } + } + + /** + * Generating source data, encoding, recovering and then verifying. + * RawErasureCoder mainly uses ECChunk to pass input and output data buffers, + * it supports two kinds of ByteBuffers, one is array backed, the other is + * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test. + * @param usingDirectBuffer + */ + protected void testCoding(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + + ErasureEncoder encoder = createEncoder(); + // Generate data and encode + ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); + // Backup all the source chunks for later recovering because some coders + // may affect the source data. + TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[]) + blockGroup.getDataBlocks()); + // Make a copy of a strip for later comparing + TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks); + + ErasureCodingStep codingStep = encoder.encode(blockGroup); + performCodingStep(codingStep); + // Erase the copied sources + eraseSomeDataBlocks(clonedDataBlocks); + + //Decode + blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks()); + ErasureDecoder decoder = createDecoder(); + codingStep = decoder.decode(blockGroup); + performCodingStep(codingStep); + + //Compare + compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks()); + } + + /** + * This is typically how a coding step should be performed. + * @param codingStep + */ + private void performCodingStep(ErasureCodingStep codingStep) { + // Pretend that we're opening these input blocks and output blocks. + ECBlock[] inputBlocks = codingStep.getInputBlocks(); + ECBlock[] outputBlocks = codingStep.getOutputBlocks(); + // We allocate input and output chunks accordingly. + ECChunk[] inputChunks = new ECChunk[inputBlocks.length]; + ECChunk[] outputChunks = new ECChunk[outputBlocks.length]; + + for (int i = 0; i < numChunksInBlock; ++i) { + // Pretend that we're reading input chunks from input blocks. + for (int j = 0; j < inputBlocks.length; ++j) { + inputChunks[j] = ((TestBlock) inputBlocks[j]).chunks[i]; + } + + // Pretend that we allocate and will write output results to the blocks. + for (int j = 0; j < outputBlocks.length; ++j) { + outputChunks[j] = allocateOutputChunk(); + ((TestBlock) outputBlocks[j]).chunks[i] = outputChunks[j]; + } + + // Given the input chunks and output chunk buffers, just call it ! + codingStep.performCoding(inputChunks, outputChunks); + } + + codingStep.finish(); + } + + /** + * Compare and verify if recovered blocks data are the same with the erased + * blocks data. + * @param erasedBlocks + * @param recoveredBlocks + */ + protected void compareAndVerify(ECBlock[] erasedBlocks, + ECBlock[] recoveredBlocks) { + for (int i = 0; i < erasedBlocks.length; ++i) { + compareAndVerify(((TestBlock) erasedBlocks[i]).chunks, + ((TestBlock) recoveredBlocks[i]).chunks); + } + } + + /** + * Create erasure encoder for test. + * @return + */ + private ErasureEncoder createEncoder() { + ErasureEncoder encoder; + try { + encoder = encoderClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed to create encoder", e); + } + + encoder.initialize(numDataUnits, numParityUnits, chunkSize); + return encoder; + } + + /** + * Create the erasure decoder for the test. + * @return + */ + private ErasureDecoder createDecoder() { + ErasureDecoder decoder; + try { + decoder = decoderClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed to create decoder", e); + } + + decoder.initialize(numDataUnits, numParityUnits, chunkSize); + return decoder; + } + + /** + * Prepare a block group for encoding. + * @return + */ + protected ECBlockGroup prepareBlockGroupForEncoding() { + ECBlock[] dataBlocks = new TestBlock[numDataUnits]; + ECBlock[] parityBlocks = new TestBlock[numParityUnits]; + + for (int i = 0; i < numDataUnits; i++) { + dataBlocks[i] = generateDataBlock(); + } + + for (int i = 0; i < numParityUnits; i++) { + parityBlocks[i] = allocateOutputBlock(); + } + + return new ECBlockGroup(dataBlocks, parityBlocks); + } + + /** + * Generate random data and return a data block. + * @return + */ + protected ECBlock generateDataBlock() { + ECChunk[] chunks = new ECChunk[numChunksInBlock]; + + for (int i = 0; i < numChunksInBlock; ++i) { + chunks[i] = generateDataChunk(); + } + + return new TestBlock(chunks); + } + + /** + * Copy those data blocks that's to be erased for later comparing and + * verifying. + * @param dataBlocks + * @return + */ + protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) { + TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length]; + + for (int i = 0; i < erasedDataIndexes.length; ++i) { + copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]); + } + + return copiedBlocks; + } + + /** + * Allocate an output block. Note the chunk buffer will be allocated by the + * up caller when performing the coding step. + * @return + */ + protected TestBlock allocateOutputBlock() { + ECChunk[] chunks = new ECChunk[numChunksInBlock]; + + return new TestBlock(chunks); + } + + /** + * Clone blocks with data copied along with, avoiding affecting the original + * blocks. + * @param blocks + * @return + */ + protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) { + TestBlock[] results = new TestBlock[blocks.length]; + for (int i = 0; i < blocks.length; ++i) { + results[i] = cloneBlockWithData(blocks[i]); + } + + return results; + } + + /** + * Clone exactly a block, avoiding affecting the original block. + * @param block + * @return a new block + */ + protected static TestBlock cloneBlockWithData(TestBlock block) { + ECChunk[] newChunks = cloneChunksWithData(block.chunks); + + return new TestBlock(newChunks); + } + + /** + * Erase some data blocks specified by the indexes from the data blocks. + * @param dataBlocks + */ + protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) { + for (int i = 0; i < erasedDataIndexes.length; ++i) { + eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]); + } + } + + /** + * Erase data from a block specified by erased index. + * @param blocks + * @param erasedIndex + */ + protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) { + TestBlock theBlock = blocks[erasedIndex]; + eraseDataFromChunks(theBlock.chunks); + theBlock.setErased(true); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java new file mode 100644 index 0000000000000..d46fe061a5c5e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java @@ -0,0 +1,50 @@ +/** + * 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.io.erasurecode.coder; + +import org.junit.Before; +import org.junit.Test; + +/** + * Test XOR encoding and decoding. + */ +public class TestXorCoder extends TestErasureCoderBase { + + @Before + public void setup() { + this.encoderClass = XorErasureEncoder.class; + this.decoderClass = XorErasureDecoder.class; + + this.numDataUnits = 10; + this.numParityUnits = 1; + this.erasedDataIndexes = new int[] {0}; + + this.numChunksInBlock = 10; + } + + @Test + public void testCodingNoDirectBuffer() { + testCoding(false); + } + + @Test + public void testCodingDirectBuffer() { + testCoding(true); + } + +} From f05c21285ef23b6a973d69f045b1cb46c5abc039 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 9 Mar 2015 14:59:58 -0700 Subject: [PATCH 016/212] HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from striped files. Contributed by Jing Zhao. --- .../hadoop/hdfs/protocol/LocatedBlock.java | 4 +- .../hdfs/protocol/LocatedStripedBlock.java | 68 +++++++++ ...amenodeProtocolServerSideTranslatorPB.java | 14 +- .../ClientNamenodeProtocolTranslatorPB.java | 13 +- ...atanodeProtocolClientSideTranslatorPB.java | 2 +- ...atanodeProtocolServerSideTranslatorPB.java | 2 +- .../hadoop/hdfs/protocolPB/PBHelper.java | 80 ++++++---- .../blockmanagement/BlockInfoStriped.java | 5 + .../BlockInfoStripedUnderConstruction.java | 99 ++++++------ .../server/blockmanagement/BlockManager.java | 46 ++++-- .../blockmanagement/DatanodeDescriptor.java | 4 +- .../blockmanagement/DatanodeStorageInfo.java | 1 + .../server/namenode/FSImageFormatPBINode.java | 21 +-- .../hdfs/server/namenode/FSNamesystem.java | 10 +- .../hadoop-hdfs/src/main/proto/hdfs.proto | 1 + .../org/apache/hadoop/hdfs/DFSTestUtil.java | 15 +- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 16 +- .../datanode/TestIncrementalBrVariations.java | 14 +- .../server/namenode/TestAddStripedBlocks.java | 141 ++++++++++++++++++ .../hdfs/server/namenode/TestFSImage.java | 5 +- 20 files changed, 428 insertions(+), 133 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java index cc13f10605e8f..4e8f2025be971 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java @@ -49,14 +49,14 @@ public class LocatedBlock { // else false. If block has few corrupt replicas, they are filtered and // their locations are not part of this object private boolean corrupt; - private Token blockToken = new Token(); + private Token blockToken = new Token<>(); /** * List of cached datanode locations */ private DatanodeInfo[] cachedLocs; // Used when there are no locations - private static final DatanodeInfoWithStorage[] EMPTY_LOCS = + static final DatanodeInfoWithStorage[] EMPTY_LOCS = new DatanodeInfoWithStorage[0]; public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java new file mode 100644 index 0000000000000..97e3a6936f840 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; + +import java.util.Arrays; + +/** + * {@link LocatedBlock} with striped block support. For a striped block, each + * datanode storage is associated with a block in the block group. We need to + * record the index (in the striped block group) for each of them. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class LocatedStripedBlock extends LocatedBlock { + private int[] blockIndices; + + public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs, + String[] storageIDs, StorageType[] storageTypes, int[] indices, + long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) { + super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs); + assert indices != null && indices.length == locs.length; + this.blockIndices = new int[indices.length]; + System.arraycopy(indices, 0, blockIndices, 0, indices.length); + } + + public LocatedStripedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages, + int[] indices, long startOffset, boolean corrupt) { + this(b, DatanodeStorageInfo.toDatanodeInfos(storages), + DatanodeStorageInfo.toStorageIDs(storages), + DatanodeStorageInfo.toStorageTypes(storages), indices, + startOffset, corrupt, EMPTY_LOCS); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + getBlock() + + "; getBlockSize()=" + getBlockSize() + + "; corrupt=" + isCorrupt() + + "; offset=" + getStartOffset() + + "; locs=" + Arrays.asList(getLocations()) + + "; indices=" + Arrays.asList(blockIndices) + + "}"; + } + + public int[] getBlockIndices() { + return this.blockIndices; + } +} 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 480b3d9bf8b11..ae5745a760cf0 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 @@ -419,7 +419,7 @@ public AppendResponseProto append(RpcController controller, req.getClientName(), flags); AppendResponseProto.Builder builder = AppendResponseProto.newBuilder(); if (result.getLastBlock() != null) { - builder.setBlock(PBHelper.convert(result.getLastBlock())); + builder.setBlock(PBHelper.convertLocatedBlock(result.getLastBlock())); } if (result.getFileStatus() != null) { builder.setStat(PBHelper.convert(result.getFileStatus())); @@ -495,7 +495,7 @@ public AddBlockResponseProto addBlock(RpcController controller, (favor == null || favor.size() == 0) ? null : favor .toArray(new String[favor.size()])); return AddBlockResponseProto.newBuilder() - .setBlock(PBHelper.convert(result)).build(); + .setBlock(PBHelper.convertLocatedBlock(result)).build(); } catch (IOException e) { throw new ServiceException(e); } @@ -519,7 +519,7 @@ public GetAdditionalDatanodeResponseProto getAdditionalDatanode( new DatanodeInfoProto[excludesList.size()])), req.getNumAdditionalNodes(), req.getClientName()); return GetAdditionalDatanodeResponseProto.newBuilder().setBlock( - PBHelper.convert(result)) + PBHelper.convertLocatedBlock(result)) .build(); } catch (IOException e) { throw new ServiceException(e); @@ -545,8 +545,8 @@ public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller, ReportBadBlocksRequestProto req) throws ServiceException { try { List bl = req.getBlocksList(); - server.reportBadBlocks(PBHelper.convertLocatedBlock( - bl.toArray(new LocatedBlockProto[bl.size()]))); + server.reportBadBlocks(PBHelper.convertLocatedBlocks( + bl.toArray(new LocatedBlockProto[bl.size()]))); } catch (IOException e) { throw new ServiceException(e); } @@ -950,8 +950,8 @@ public UpdateBlockForPipelineResponseProto updateBlockForPipeline( RpcController controller, UpdateBlockForPipelineRequestProto req) throws ServiceException { try { - LocatedBlockProto result = PBHelper.convert(server - .updateBlockForPipeline(PBHelper.convert(req.getBlock()), + LocatedBlockProto result = PBHelper.convertLocatedBlock( + server.updateBlockForPipeline(PBHelper.convert(req.getBlock()), req.getClientName())); return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result) .build(); 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 4ec6f9ea05e94..22da083c98882 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 @@ -327,7 +327,7 @@ public LastBlockWithStatus append(String src, String clientName, try { AppendResponseProto res = rpcProxy.append(null, req); LocatedBlock lastBlock = res.hasBlock() ? PBHelper - .convert(res.getBlock()) : null; + .convertLocatedBlockProto(res.getBlock()) : null; HdfsFileStatus stat = (res.hasStat()) ? PBHelper.convert(res.getStat()) : null; return new LastBlockWithStatus(lastBlock, stat); @@ -415,7 +415,8 @@ public LocatedBlock addBlock(String src, String clientName, req.addAllFavoredNodes(Arrays.asList(favoredNodes)); } try { - return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock()); + return PBHelper.convertLocatedBlockProto( + rpcProxy.addBlock(null, req.build()).getBlock()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } @@ -440,8 +441,8 @@ public LocatedBlock getAdditionalDatanode(String src, long fileId, .setClientName(clientName) .build(); try { - return PBHelper.convert(rpcProxy.getAdditionalDatanode(null, req) - .getBlock()); + return PBHelper.convertLocatedBlockProto( + rpcProxy.getAdditionalDatanode(null, req).getBlock()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } @@ -468,7 +469,7 @@ public boolean complete(String src, String clientName, @Override public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder() - .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlock(blocks))) + .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlocks(blocks))) .build(); try { rpcProxy.reportBadBlocks(null, req); @@ -900,7 +901,7 @@ public LocatedBlock updateBlockForPipeline(ExtendedBlock block, .setClientName(clientName) .build(); try { - return PBHelper.convert( + return PBHelper.convertLocatedBlockProto( rpcProxy.updateBlockForPipeline(null, req).getBlock()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); 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 825e83586b323..2db70d19bd722 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 @@ -279,7 +279,7 @@ public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto .newBuilder(); for (int i = 0; i < blocks.length; i++) { - builder.addBlocks(i, PBHelper.convert(blocks[i])); + builder.addBlocks(i, PBHelper.convertLocatedBlock(blocks[i])); } ReportBadBlocksRequestProto req = builder.build(); try { 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 873eb6d1708e7..19b2f83ce3e71 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 @@ -258,7 +258,7 @@ public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller, List lbps = request.getBlocksList(); LocatedBlock [] blocks = new LocatedBlock [lbps.size()]; for(int i=0; i locs = proto.getLocsList(); DatanodeInfo[] targets = new DatanodeInfo[locs.size()]; @@ -831,6 +838,15 @@ public static LocatedBlock convert(LocatedBlockProto proto) { storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]); } + int[] indices = null; + final int indexCount = proto.getBlockIndexCount(); + if (indexCount > 0) { + indices = new int[indexCount]; + for (int i = 0; i < indexCount; i++) { + indices[i] = proto.getBlockIndex(i); + } + } + // Set values from the isCached list, re-using references from loc List cachedLocs = new ArrayList(locs.size()); List isCachedList = proto.getIsCachedList(); @@ -840,9 +856,17 @@ public static LocatedBlock convert(LocatedBlockProto proto) { } } - LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, - storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(), - cachedLocs.toArray(new DatanodeInfo[0])); + final LocatedBlock lb; + if (indices == null) { + lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, storageIDs, + storageTypes, proto.getOffset(), proto.getCorrupt(), + cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()])); + } else { + lb = new LocatedStripedBlock(PBHelper.convert(proto.getB()), targets, + storageIDs, storageTypes, indices, proto.getOffset(), + proto.getCorrupt(), + cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()])); + } lb.setBlockToken(PBHelper.convert(proto.getBlockToken())); return lb; @@ -1258,36 +1282,36 @@ public static NamespaceInfoProto convert(NamespaceInfo info) { } // Located Block Arrays and Lists - public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) { + public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) { if (lb == null) return null; - return convertLocatedBlock2(Arrays.asList(lb)).toArray( - new LocatedBlockProto[lb.length]); + return convertLocatedBlocks2(Arrays.asList(lb)) + .toArray(new LocatedBlockProto[lb.length]); } - public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) { + public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) { if (lb == null) return null; - return convertLocatedBlock(Arrays.asList(lb)).toArray( - new LocatedBlock[lb.length]); + return convertLocatedBlocks(Arrays.asList(lb)) + .toArray(new LocatedBlock[lb.length]); } - public static List convertLocatedBlock( + public static List convertLocatedBlocks( List lb) { if (lb == null) return null; final int len = lb.size(); - List result = - new ArrayList(len); - for (int i = 0; i < len; ++i) { - result.add(PBHelper.convert(lb.get(i))); + List result = new ArrayList<>(len); + for (LocatedBlockProto aLb : lb) { + result.add(PBHelper.convertLocatedBlockProto(aLb)); } return result; } - public static List convertLocatedBlock2(List lb) { + public static List convertLocatedBlocks2( + List lb) { if (lb == null) return null; final int len = lb.size(); - List result = new ArrayList(len); - for (int i = 0; i < len; ++i) { - result.add(PBHelper.convert(lb.get(i))); + List result = new ArrayList<>(len); + for (LocatedBlock aLb : lb) { + result.add(PBHelper.convertLocatedBlock(aLb)); } return result; } @@ -1297,8 +1321,9 @@ public static List convertLocatedBlock2(List lb public static LocatedBlocks convert(LocatedBlocksProto lb) { return new LocatedBlocks( lb.getFileLength(), lb.getUnderConstruction(), - PBHelper.convertLocatedBlock(lb.getBlocksList()), - lb.hasLastBlock() ? PBHelper.convert(lb.getLastBlock()) : null, + PBHelper.convertLocatedBlocks(lb.getBlocksList()), + lb.hasLastBlock() ? + PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null, lb.getIsLastBlockComplete(), lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null); @@ -1311,14 +1336,15 @@ public static LocatedBlocksProto convert(LocatedBlocks lb) { LocatedBlocksProto.Builder builder = LocatedBlocksProto.newBuilder(); if (lb.getLastLocatedBlock() != null) { - builder.setLastBlock(PBHelper.convert(lb.getLastLocatedBlock())); + builder.setLastBlock( + PBHelper.convertLocatedBlock(lb.getLastLocatedBlock())); } if (lb.getFileEncryptionInfo() != null) { builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo())); } return builder.setFileLength(lb.getFileLength()) .setUnderConstruction(lb.isUnderConstruction()) - .addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks())) + .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks())) .setIsLastBlockComplete(lb.isLastBlockComplete()).build(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 8b458df82dceb..84c3be60e8fd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -123,6 +123,11 @@ private int findStorageInfoFromEnd(DatanodeStorageInfo storage) { return -1; } + int getStorageBlockIndex(DatanodeStorageInfo storage) { + int i = this.findStorageInfo(storage); + return i == -1 ? -1 : indices[i]; + } + @Override boolean removeStorage(DatanodeStorageInfo storage) { int dnIndex = findStorageInfoFromEnd(storage); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index 151241b25efde..b1857bbb88a4e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -23,9 +23,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION; @@ -39,12 +36,8 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped { /** * Block replicas as assigned when the block was allocated. - * - * TODO: we need to update this attribute, along with the return type of - * getExpectedStorageLocations and LocatedBlock. For striped blocks, clients - * need to understand the index of each striped block in the block group. */ - private List replicas; + private ReplicaUnderConstruction[] replicas; /** * The new generation stamp, which this block will have @@ -75,12 +68,12 @@ assert getBlockUCState() != COMPLETE : /** * Convert an under construction striped block to a complete striped block. - * + * * @return BlockInfoStriped - a complete block. - * @throws IOException if the state of the block - * (the generation stamp and the length) has not been committed by - * the client or it does not have at least a minimal number of replicas - * reported from data-nodes. + * @throws IOException if the state of the block + * (the generation stamp and the length) has not been committed by + * the client or it does not have at least a minimal number of replicas + * reported from data-nodes. */ BlockInfoStriped convertToCompleteBlock() throws IOException { assert getBlockUCState() != COMPLETE : @@ -91,10 +84,13 @@ assert getBlockUCState() != COMPLETE : /** Set expected locations */ public void setExpectedLocations(DatanodeStorageInfo[] targets) { int numLocations = targets == null ? 0 : targets.length; - this.replicas = new ArrayList<>(numLocations); + this.replicas = new ReplicaUnderConstruction[numLocations]; for(int i = 0; i < numLocations; i++) { - replicas.add(new ReplicaUnderConstruction(this, targets[i], - ReplicaState.RBW)); + // when creating a new block we simply sequentially assign block index to + // each storage + Block blk = new Block(this.getBlockId() + i, this.getGenerationStamp(), 0); + replicas[i] = new ReplicaUnderConstruction(blk, targets[i], + ReplicaState.RBW); } } @@ -106,14 +102,24 @@ public DatanodeStorageInfo[] getExpectedStorageLocations() { int numLocations = getNumExpectedLocations(); DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; for (int i = 0; i < numLocations; i++) { - storages[i] = replicas.get(i).getExpectedStorageLocation(); + storages[i] = replicas[i].getExpectedStorageLocation(); } return storages; } + /** @return the index array indicating the block index in each storage */ + public int[] getBlockIndices() { + int numLocations = getNumExpectedLocations(); + int[] indices = new int[numLocations]; + for (int i = 0; i < numLocations; i++) { + indices[i] = BlockIdManager.getBlockIndex(replicas[i]); + } + return indices; + } + /** Get the number of expected locations */ public int getNumExpectedLocations() { - return replicas == null ? 0 : replicas.size(); + return replicas == null ? 0 : replicas.length; } /** @@ -178,7 +184,7 @@ void commitBlock(Block block) throws IOException { public void initializeBlockRecovery(long recoveryId) { setBlockUCState(BlockUCState.UNDER_RECOVERY); blockRecoveryId = recoveryId; - if (replicas == null || replicas.size() == 0) { + if (replicas == null || replicas.length == 0) { NameNode.blockStateChangeLog.warn("BLOCK*" + " BlockInfoUnderConstruction.initLeaseRecovery:" + " No blocks found, lease removed."); @@ -186,28 +192,36 @@ public void initializeBlockRecovery(long recoveryId) { // TODO we need to implement different recovery logic here } - void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block, + void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock, ReplicaState rState) { - Iterator it = replicas.iterator(); - while (it.hasNext()) { - ReplicaUnderConstruction r = it.next(); - DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation(); - if (expectedLocation == storage) { - // Record the gen stamp from the report - r.setGenerationStamp(block.getGenerationStamp()); - return; - } else if (expectedLocation != null && - expectedLocation.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; + if (replicas == null) { + replicas = new ReplicaUnderConstruction[1]; + replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState); + } else { + for (int i = 0; i < replicas.length; i++) { + DatanodeStorageInfo expected = replicas[i].getExpectedStorageLocation(); + if (expected == storage) { + replicas[i].setBlockId(reportedBlock.getBlockId()); + replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp()); + return; + } else if (expected != null && expected.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. + replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage, + rState); + return; + } } + ReplicaUnderConstruction[] newReplicas = + new ReplicaUnderConstruction[replicas.length + 1]; + System.arraycopy(replicas, 0, newReplicas, 0, replicas.length); + newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction( + reportedBlock, storage, rState); + replicas = newReplicas; } - replicas.add(new ReplicaUnderConstruction(block, storage, rState)); } @Override @@ -226,12 +240,11 @@ public void appendStringTo(StringBuilder sb) { private void appendUCParts(StringBuilder sb) { sb.append("{UCState=").append(blockUCState).append(", replicas=["); if (replicas != null) { - Iterator iter = replicas.iterator(); - if (iter.hasNext()) { - iter.next().appendStringTo(sb); - while (iter.hasNext()) { + int i = 0; + for (ReplicaUnderConstruction r : replicas) { + r.appendStringTo(sb); + if (++i < replicas.length) { sb.append(", "); - iter.next().appendStringTo(sb); } } } 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 70e697e7e4516..35a4e3e171da2 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 @@ -55,6 +55,7 @@ import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; @@ -841,13 +842,24 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos, } /** @return a LocatedBlock for the given block */ - private LocatedBlock createLocatedBlock(final BlockInfo blk, - final long pos) throws IOException { - if (blk instanceof BlockInfoContiguousUnderConstruction) { - if (blk.isComplete()) { - throw new IOException( - "blk instanceof BlockInfoUnderConstruction && blk.isComplete()" - + ", blk=" + blk); + private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) { + if (!blk.isComplete()) { + if (blk.isStriped()) { + final BlockInfoStripedUnderConstruction uc = + (BlockInfoStripedUnderConstruction) blk; + final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); + final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), + blk); + return new LocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, + false); + } else { + assert blk instanceof BlockInfoContiguousUnderConstruction; + final BlockInfoContiguousUnderConstruction uc = + (BlockInfoContiguousUnderConstruction) blk; + final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); + final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), + blk); + return new LocatedBlock(eb, storages, pos, false); } final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction) blk; @@ -855,7 +867,6 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); return newLocatedBlock(eb, storages, pos, false); } - // TODO support BlockInfoStripedUC // get block locations final int numCorruptNodes = countNodes(blk).corruptReplicas(); @@ -871,13 +882,21 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, numCorruptNodes == numNodes; final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes; final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines]; - int j = 0; + final int[] blockIndices = blk.isStriped() ? new int[numMachines] : null; + int j = 0, i = 0; if (numMachines > 0) { for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) { final DatanodeDescriptor d = storage.getDatanodeDescriptor(); final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d); - if (isCorrupt || (!replicaCorrupt)) + if (isCorrupt || (!replicaCorrupt)) { machines[j++] = storage; + // TODO this can be more efficient + if (blockIndices != null) { + int index = ((BlockInfoStriped) blk).getStorageBlockIndex(storage); + assert index >= 0; + blockIndices[i++] = index; + } + } } } assert j == machines.length : @@ -887,7 +906,9 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, " numCorrupt: " + numCorruptNodes + " numCorruptRepls: " + numCorruptReplicas; final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return newLocatedBlock(eb, machines, pos, isCorrupt); + return blockIndices == null ? + newLocatedBlock(eb, machines, pos, isCorrupt) : + new LocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt); } /** Create a LocatedBlocks. */ @@ -2501,7 +2522,8 @@ private boolean isBlockUnderConstruction(BlockInfo storedBlock, void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, DatanodeStorageInfo storageInfo) throws IOException { BlockInfo block = ucBlock.storedBlock; - BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState); + BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, + ucBlock.reportedState); if (ucBlock.reportedState == ReplicaState.FINALIZED && (block.findStorageInfo(storageInfo) < 0)) { 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 415646a5ec9e1..b7a3489b08957 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 @@ -276,7 +276,9 @@ public DatanodeStorageInfo getStorageInfo(String storageID) { return storageMap.get(storageID); } } - DatanodeStorageInfo[] getStorageInfos() { + + @VisibleForTesting + public DatanodeStorageInfo[] getStorageInfos() { synchronized (storageMap) { final Collection storages = storageMap.values(); return storages.toArray(new DatanodeStorageInfo[storages.size()]); 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 index 2c8b3eaba708e..9b5a923862bdd 100644 --- 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 @@ -206,6 +206,7 @@ boolean areBlocksOnFailedStorage() { return getState() == State.FAILED && numBlocks != 0; } + @VisibleForTesting public String getStorageID() { return storageID; } 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 a58e37ef6bcd9..82398ee78bc75 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 @@ -366,16 +366,19 @@ private INodeFile loadINodeFile(INodeSection.INode n) { INodeSection.FileUnderConstructionFeature uc = f.getFileUC(); file.toUnderConstruction(uc.getClientName(), uc.getClientMachine()); BlockInfo lastBlk = file.getLastBlock(); - // replace the last block of file - final BlockInfo ucBlk; - if (stripeFeature != null) { - BlockInfoStriped striped = (BlockInfoStriped) lastBlk; - ucBlk = new BlockInfoStripedUnderConstruction(striped, - striped.getDataBlockNum(), striped.getParityBlockNum()); - } else { - ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication); + if (lastBlk != null) { + // replace the last block of file + final BlockInfo ucBlk; + if (stripeFeature != null) { + BlockInfoStriped striped = (BlockInfoStriped) lastBlk; + ucBlk = new BlockInfoStripedUnderConstruction(striped, + striped.getDataBlockNum(), striped.getParityBlockNum()); + } else { + ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, + replication); + } + file.setBlock(file.numBlocks() - 1, ucBlk); } - file.setBlock(file.numBlocks() - 1, ucBlk); } return file; } 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 fca848e27a68f..a3b32c5521d7c 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 @@ -189,6 +189,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.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.protocol.RollingUpgradeException; @@ -206,6 +207,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; @@ -1780,8 +1782,14 @@ LocatedBlocks getBlockLocations(String clientMachine, String srcArg, LocatedBlocks blocks = res.blocks; if (blocks != null) { + List blkList = blocks.getLocatedBlocks(); + if (blkList == null || blkList.size() == 0 || + blkList.get(0) instanceof LocatedStripedBlock) { + // no need to sort locations for striped blocks + return blocks; + } blockManager.getDatanodeManager().sortLocatedBlocks( - clientMachine, blocks.getLocatedBlocks()); + clientMachine, blkList); // lastBlock is not part of getLocatedBlocks(), might need to sort it too LocatedBlock lastBlock = blocks.getLastLocatedBlock(); 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 d281a64538a9c..67e20582f2230 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -220,6 +220,7 @@ message LocatedBlockProto { repeated bool isCached = 6 [packed=true]; // if a location in locs is cached repeated StorageTypeProto storageTypes = 7; repeated string storageIDs = 8; + repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage } message DataEncryptionKeyProto { 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 c6d44d19d3082..53a99b05eb931 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 @@ -109,7 +109,6 @@ 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.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; @@ -133,6 +132,10 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; +import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.nativeio.NativeIO; @@ -1811,7 +1814,7 @@ public static void resetLastUpdatesWithOffset(DatanodeInfo dn, long offset) { dn.setLastUpdate(Time.now() + offset); dn.setLastUpdateMonotonic(Time.monotonicNow() + offset); } - + /** * This method takes a set of block locations and fills the provided buffer * with expected bytes based on simulated content from @@ -1835,4 +1838,12 @@ public static void fillExpectedBuf(LocatedBlocks lbs, byte[] expected) { } } + public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock( + Block block, BlockStatus blockStatus, DatanodeStorage storage) { + ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1]; + receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, blockStatus, null); + StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1]; + reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks); + return reports; + } } 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 c7233bd5da9a6..4b42f4cd3e977 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 @@ -489,16 +489,16 @@ private LocatedBlock createLocatedBlockNoStorageMedia() { @Test public void testConvertLocatedBlock() { LocatedBlock lb = createLocatedBlock(); - LocatedBlockProto lbProto = PBHelper.convert(lb); - LocatedBlock lb2 = PBHelper.convert(lbProto); + LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb); + LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto); compare(lb,lb2); } @Test public void testConvertLocatedBlockNoStorageMedia() { LocatedBlock lb = createLocatedBlockNoStorageMedia(); - LocatedBlockProto lbProto = PBHelper.convert(lb); - LocatedBlock lb2 = PBHelper.convert(lbProto); + LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb); + LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto); compare(lb,lb2); } @@ -508,8 +508,8 @@ public void testConvertLocatedBlockList() { for (int i=0;i<3;i++) { lbl.add(createLocatedBlock()); } - List lbpl = PBHelper.convertLocatedBlock2(lbl); - List lbl2 = PBHelper.convertLocatedBlock(lbpl); + List lbpl = PBHelper.convertLocatedBlocks2(lbl); + List lbl2 = PBHelper.convertLocatedBlocks(lbpl); assertEquals(lbl.size(), lbl2.size()); for (int i=0;i storageIDs = new ArrayList<>(); + // create an empty file + FSDataOutputStream out = null; + try { + out = dfs.create(file, (short) 1); + + // 1. create the UC striped block + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + cluster.getNamesystem().getAdditionalBlock(file.toString(), + fileNode.getId(), dfs.getClient().getClientName(), null, null, null); + BlockInfo lastBlock = fileNode.getLastBlock(); + BlockInfoStripedUnderConstruction ucBlock = + (BlockInfoStripedUnderConstruction) lastBlock; + + DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); + int[] indices = ucBlock.getBlockIndices(); + Assert.assertEquals(GROUP_SIZE, locs.length); + Assert.assertEquals(GROUP_SIZE, indices.length); + + // 2. mimic incremental block reports and make sure the uc-replica list in + // the BlockStripedUC is correct + int i = 0; + for (DataNode dn : cluster.getDataNodes()) { + final Block block = new Block(lastBlock.getBlockId() + i++, + lastBlock.getGenerationStamp(), 0); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + storageIDs.add(storage.getStorageID()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, BlockStatus.RECEIVING_BLOCK, + storage); + for (StorageReceivedDeletedBlocks report : reports) { + cluster.getNamesystem().processIncrementalBlockReport( + dn.getDatanodeId(), report); + } + } + + // make sure lastBlock is correct and the storages have been updated + locs = ucBlock.getExpectedStorageLocations(); + indices = ucBlock.getBlockIndices(); + Assert.assertEquals(GROUP_SIZE, locs.length); + Assert.assertEquals(GROUP_SIZE, indices.length); + for (DatanodeStorageInfo newstorage : locs) { + Assert.assertTrue(storageIDs.contains(newstorage.getStorageID())); + } + } finally { + IOUtils.cleanup(null, out); + } + + // 3. restart the namenode. mimic the full block reports and check the + // uc-replica list again + cluster.restartNameNode(true); + final String bpId = cluster.getNamesystem().getBlockPoolId(); + INodeFile fileNode = cluster.getNamesystem().getFSDirectory() + .getINode4Write(file.toString()).asFile(); + BlockInfo lastBlock = fileNode.getLastBlock(); + int i = GROUP_SIZE - 1; + for (DataNode dn : cluster.getDataNodes()) { + String storageID = storageIDs.get(i); + final Block block = new Block(lastBlock.getBlockId() + i--, + lastBlock.getGenerationStamp(), 0); + DatanodeStorage storage = new DatanodeStorage(storageID); + List blocks = new ArrayList<>(); + ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null, + null); + blocks.add(replica); + BlockListAsLongs bll = new BlockListAsLongs(null, blocks); + StorageBlockReport[] reports = {new StorageBlockReport(storage, + bll.getBlockListAsLongs())}; + cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId), + bpId, reports); + } + + BlockInfoStripedUnderConstruction ucBlock = + (BlockInfoStripedUnderConstruction) lastBlock; + DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); + int[] indices = ucBlock.getBlockIndices(); + Assert.assertEquals(GROUP_SIZE, locs.length); + Assert.assertEquals(GROUP_SIZE, indices.length); + for (i = 0; i < GROUP_SIZE; i++) { + Assert.assertEquals(storageIDs.get(i), + locs[GROUP_SIZE - 1 - i].getStorageID()); + Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]); + } + } } 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 8a87ed3bbccce..25c0bcfc04fd8 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 @@ -25,6 +25,8 @@ import java.util.EnumSet; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.junit.Assert; import org.apache.hadoop.conf.Configuration; @@ -219,8 +221,7 @@ public void testZeroBlockSize() throws Exception { .format(false) .manageDataDfsDirs(false) .manageNameDfsDirs(false) - .waitSafeMode(false) - .startupOption(StartupOption.UPGRADE) + .waitSafeMode(false).startupOption(StartupOption.UPGRADE) .build(); try { FileSystem fs = cluster.getFileSystem(); From 292e367d0772f47e434de4a31af4edf1b07241dc Mon Sep 17 00:00:00 2001 From: drankye Date: Thu, 12 Mar 2015 23:35:22 +0800 Subject: [PATCH 017/212] HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 4 +++ .../coder/AbstractErasureCoder.java | 5 ++- .../rawcoder/AbstractRawErasureCoder.java | 5 ++- .../hadoop/io/erasurecode/TestCoderBase.java | 6 ++++ .../coder/TestErasureCoderBase.java | 36 ++++++++++++++++--- .../rawcoder/TestRawCoderBase.java | 13 +++++-- 6 files changed, 60 insertions(+), 9 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index c17a1bd9037a0..a97dc34b87bc4 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -18,3 +18,7 @@ HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Kai Zheng via vinayakumarb ) + HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng + ( Kai Zheng ) + + diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index f2cc041e0b678..8d3bc3405041d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -17,12 +17,15 @@ */ package org.apache.hadoop.io.erasurecode.coder; +import org.apache.hadoop.conf.Configured; + /** * A common class of basic facilities to be shared by encoder and decoder * * It implements the {@link ErasureCoder} interface. */ -public abstract class AbstractErasureCoder implements ErasureCoder { +public abstract class AbstractErasureCoder + extends Configured implements ErasureCoder { private int numDataUnits; private int numParityUnits; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index 74d2ab6fc1ecf..e6f3d929daadf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -17,12 +17,15 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; +import org.apache.hadoop.conf.Configured; + /** * A common class of basic facilities to be shared by encoder and decoder * * It implements the {@link RawErasureCoder} interface. */ -public abstract class AbstractRawErasureCoder implements RawErasureCoder { +public abstract class AbstractRawErasureCoder + extends Configured implements RawErasureCoder { private int numDataUnits; private int numParityUnits; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 3c4288c8c5990..194413a43ac49 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -43,6 +43,12 @@ public abstract class TestCoderBase { // may go to different coding implementations. protected boolean usingDirectBuffer = true; + /** + * Prepare before running the case. + * @param numDataUnits + * @param numParityUnits + * @param erasedIndexes + */ protected void prepare(int numDataUnits, int numParityUnits, int[] erasedIndexes) { this.numDataUnits = numDataUnits; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index ca5c1c9c5356d..36e061a3ee5cb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.coder; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.ECBlockGroup; @@ -29,6 +30,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase { protected Class encoderClass; protected Class decoderClass; + private Configuration conf; protected int numChunksInBlock = 16; /** @@ -45,6 +47,19 @@ public TestBlock(ECChunk[] chunks) { } } + /** + * Prepare before running the case. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @param erasedIndexes + */ + protected void prepare(Configuration conf, int numDataUnits, + int numParityUnits, int[] erasedIndexes) { + this.conf = conf; + super.prepare(numDataUnits, numParityUnits, erasedIndexes); + } + /** * Generating source data, encoding, recovering and then verifying. * RawErasureCoder mainly uses ECChunk to pass input and output data buffers, @@ -56,6 +71,7 @@ protected void testCoding(boolean usingDirectBuffer) { this.usingDirectBuffer = usingDirectBuffer; ErasureEncoder encoder = createEncoder(); + // Generate data and encode ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); // Backup all the source chunks for later recovering because some coders @@ -65,17 +81,25 @@ protected void testCoding(boolean usingDirectBuffer) { // Make a copy of a strip for later comparing TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks); - ErasureCodingStep codingStep = encoder.encode(blockGroup); - performCodingStep(codingStep); + ErasureCodingStep codingStep; + try { + codingStep = encoder.encode(blockGroup); + performCodingStep(codingStep); + } finally { + encoder.release(); + } // Erase the copied sources eraseSomeDataBlocks(clonedDataBlocks); //Decode blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks()); ErasureDecoder decoder = createDecoder(); - codingStep = decoder.decode(blockGroup); - performCodingStep(codingStep); - + try { + codingStep = decoder.decode(blockGroup); + performCodingStep(codingStep); + } finally { + decoder.release(); + } //Compare compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks()); } @@ -138,6 +162,7 @@ private ErasureEncoder createEncoder() { } encoder.initialize(numDataUnits, numParityUnits, chunkSize); + encoder.setConf(conf); return encoder; } @@ -154,6 +179,7 @@ private ErasureDecoder createDecoder() { } decoder.initialize(numDataUnits, numParityUnits, chunkSize); + decoder.setConf(conf); return decoder; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 5f6ccda23893b..890f632370f12 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -49,7 +49,11 @@ protected void testCoding(boolean usingDirectBuffer) { // Make a copy of a strip for later comparing ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks); - encoder.encode(dataChunks, parityChunks); + try { + encoder.encode(dataChunks, parityChunks); + } finally { + encoder.release(); + } // Erase the copied sources eraseSomeDataBlocks(clonedDataChunks); @@ -58,7 +62,12 @@ protected void testCoding(boolean usingDirectBuffer) { parityChunks); ECChunk[] recoveredChunks = prepareOutputChunksForDecoding(); RawErasureDecoder decoder = createDecoder(); - decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks); + try { + decoder.decode(inputChunks, + getErasedIndexesForDecoding(), recoveredChunks); + } finally { + decoder.release(); + } //Compare compareAndVerify(toEraseDataChunks, recoveredChunks); From 68caf8728e35546aed30158e3006f37d41303397 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 13 Mar 2015 00:13:06 +0800 Subject: [PATCH 018/212] Fixed a compiling issue introduced by HADOOP-11705. --- .../hadoop/io/erasurecode/coder/TestErasureCoderBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index 36e061a3ee5cb..d911db9e19921 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -162,7 +162,7 @@ private ErasureEncoder createEncoder() { } encoder.initialize(numDataUnits, numParityUnits, chunkSize); - encoder.setConf(conf); + ((AbstractErasureCoder)encoder).setConf(conf); return encoder; } @@ -179,7 +179,7 @@ private ErasureDecoder createDecoder() { } decoder.initialize(numDataUnits, numParityUnits, chunkSize); - decoder.setConf(conf); + ((AbstractErasureCoder)decoder).setConf(conf); return decoder; } From 11585883a9eb30ba080b9aa49dba42cb0a797d75 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 16 Mar 2015 14:27:21 -0700 Subject: [PATCH 019/212] HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903 and HDFS-7435. Contributed by Zhe Zhang. --- .../hdfs/server/blockmanagement/DecommissionManager.java | 2 +- .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 2 +- .../hadoop/hdfs/server/namenode/snapshot/FileDiffList.java | 3 ++- .../test/java/org/apache/hadoop/hdfs/TestDecommission.java | 5 ++--- .../hadoop/hdfs/server/namenode/TestAddStripedBlocks.java | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java index 8a3657611c123..dc697f06436af 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java @@ -535,7 +535,7 @@ private AbstractList handleInsufficientlyReplicated( */ private void processBlocksForDecomInternal( final DatanodeDescriptor datanode, - final Iterator it, + final Iterator it, final List insufficientlyReplicated, boolean pruneSufficientlyReplicated) { boolean firstReplicationLog = true; 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 a3b32c5521d7c..b9ce89e73553b 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 @@ -2030,7 +2030,7 @@ boolean truncateInternal(String src, long newLength, } // 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(); 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 b722aa9f07cba..a5c782ddafd8b 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,6 +21,7 @@ import java.util.List; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -131,7 +132,7 @@ void combineAndCollectSnapshotBlocks( 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)) { 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 1ab7427f97a61..b23305712c70d 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.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,8 +671,7 @@ public void testRecommission() throws Exception { GenericTestUtils.waitFor(new Supplier() { @Override public Boolean get() { - BlockInfoContiguous info = - blockManager.getStoredBlock(b.getLocalBlock()); + BlockInfo info = blockManager.getStoredBlock(b.getLocalBlock()); int count = 0; StringBuilder sb = new StringBuilder("Replica locations: "); for (int i = 0; i < info.numNodes(); i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 8b513091adcf6..05aec4be95b02 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -265,9 +265,9 @@ public void testAddUCReplica() throws Exception { ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null, null); blocks.add(replica); - BlockListAsLongs bll = new BlockListAsLongs(null, blocks); + BlockListAsLongs bll = BlockListAsLongs.encode(blocks); StorageBlockReport[] reports = {new StorageBlockReport(storage, - bll.getBlockListAsLongs())}; + bll)}; cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId), bpId, reports); } From 26773d9d6c10479982a3cdbea3a0933f4476add3 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 16 Mar 2015 16:37:08 -0700 Subject: [PATCH 020/212] HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki. --- .../blockmanagement/BlockInfoStriped.java | 12 +++- .../hdfs/server/namenode/INodeFile.java | 61 +++++++++++++++---- 2 files changed, 60 insertions(+), 13 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 84c3be60e8fd1..cef83185b168d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; /** @@ -34,6 +35,7 @@ * array to record the block index for each triplet. */ public class BlockInfoStriped extends BlockInfo { + private final int chunkSize = HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; private final short dataBlockNum; private final short parityBlockNum; /** @@ -56,7 +58,7 @@ public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) { this.setBlockCollection(b.getBlockCollection()); } - short getTotalBlockNum() { + public short getTotalBlockNum() { return (short) (dataBlockNum + parityBlockNum); } @@ -178,6 +180,14 @@ void replaceBlock(BlockInfo newBlock) { } } + public long spaceConsumed() { + // In case striped blocks, total usage by this striped blocks should + // be the total of data blocks and parity blocks because + // `getNumBytes` is the total of actual data block size. + return ((getNumBytes() - 1) / (dataBlockNum * chunkSize) + 1) + * chunkSize * parityBlockNum + getNumBytes(); + } + @Override public final boolean isStriped() { return true; 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 31562f28f0248..b2c4ae8ef4330 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 @@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff; @@ -676,6 +677,11 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, final long ssDeltaNoReplication; short replication; + + if (isStriped()) { + return computeQuotaUsageWithStriped(bsps, counts); + } + if (last < lastSnapshotId) { ssDeltaNoReplication = computeFileSize(true, false); replication = getFileReplication(); @@ -698,6 +704,23 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, return counts; } + /** + * Compute quota of striped file + * @param bsps + * @param counts + * @param useCache + * @param lastSnapshotId + * @return quota counts + */ + public final QuotaCounts computeQuotaUsageWithStriped( + BlockStoragePolicySuite bsps, QuotaCounts counts) { + long nsDelta = 1; + final long ssDelta = storagespaceConsumed(); + counts.addNameSpace(nsDelta); + counts.addStorageSpace(ssDelta); + return counts; + } + @Override public final ContentSummaryComputationContext computeContentSummary( final ContentSummaryComputationContext summary) { @@ -776,23 +799,37 @@ public final long computeFileSizeNotIncludingLastUcBlock() { * @return file size */ public final long computeFileSize(boolean includesLastUcBlock, - boolean usePreferredBlockSize4LastUcBlock) { - if (blocks == null || blocks.length == 0) { + boolean usePreferredBlockSize4LastUcBlock) { + BlockInfo[] blockInfos = getBlocks(); + // In case of contiguous blocks + if (blockInfos == null || blockInfos.length == 0) { return 0; } - final int last = blocks.length - 1; + final int last = blockInfos.length - 1; //check if the last block is BlockInfoUnderConstruction - long size = blocks[last].getNumBytes(); - if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) { - if (!includesLastUcBlock) { - size = 0; - } else if (usePreferredBlockSize4LastUcBlock) { - size = getPreferredBlockSize(); - } + long size = blockInfos[last].getNumBytes(); + if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) { + if (!includesLastUcBlock) { + size = 0; + } else if (usePreferredBlockSize4LastUcBlock) { + size = getPreferredBlockSize(); + } + } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) { + if (!includesLastUcBlock) { + size = 0; + } else if (usePreferredBlockSize4LastUcBlock) { + // Striped blocks keeps block group which counts + // (data blocks num + parity blocks num). When you + // count actual used size by BlockInfoStripedUC must + // be multiplied by these blocks number. + BlockInfoStripedUnderConstruction blockInfoStripedUC + = (BlockInfoStripedUnderConstruction) blockInfos[last]; + size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum(); + } } //sum other blocks - for(int i = 0; i < last; i++) { - size += blocks[i].getNumBytes(); + for (int i = 0; i < last; i++) { + size += blockInfos[i].getNumBytes(); } return size; } From a38a37c63417a3b19dcdf98251af196c9d7b8c31 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 17 Mar 2015 10:18:50 -0700 Subject: [PATCH 021/212] HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao. --- .../server/blockmanagement/BlockManager.java | 47 ++++++------ .../PendingReplicationBlocks.java | 51 ++++++------- .../UnderReplicatedBlocks.java | 49 ++++++------ .../hdfs/server/namenode/FSDirAttrOp.java | 10 ++- .../hdfs/server/namenode/FSNamesystem.java | 21 +++--- .../hadoop/hdfs/server/namenode/INode.java | 10 +-- .../hdfs/server/namenode/INodeFile.java | 4 +- .../hdfs/server/namenode/NamenodeFsck.java | 58 ++++++++------- .../hadoop/hdfs/server/namenode/SafeMode.java | 3 +- .../blockmanagement/BlockManagerTestUtil.java | 5 +- .../blockmanagement/TestBlockManager.java | 8 +- .../server/blockmanagement/TestNodeCount.java | 3 +- .../TestOverReplicatedBlocks.java | 5 +- .../TestPendingReplication.java | 20 +++-- .../TestRBWBlockInvalidation.java | 4 +- .../TestReplicationPolicy.java | 74 +++++++++++-------- .../TestUnderReplicatedBlockQueues.java | 16 ++-- .../datanode/TestReadOnlySharedStorage.java | 9 ++- .../namenode/TestProcessCorruptBlocks.java | 5 +- 19 files changed, 220 insertions(+), 182 deletions(-) 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 35a4e3e171da2..7d4139c7376bc 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 @@ -1346,7 +1346,7 @@ int computeInvalidateWork(int nodesToProcess) { * @return number of blocks scheduled for replication during this iteration. */ int computeReplicationWork(int blocksToProcess) { - List> blocksToReplicate = null; + List> blocksToReplicate = null; namesystem.writeLock(); try { // Choose the blocks to be replicated @@ -1364,7 +1364,7 @@ int computeReplicationWork(int blocksToProcess) { * @return the number of blocks scheduled for replication */ @VisibleForTesting - int computeReplicationWorkForBlocks(List> blocksToReplicate) { + int computeReplicationWorkForBlocks(List> blocksToReplicate) { int requiredReplication, numEffectiveReplicas; List containingNodes; DatanodeDescriptor srcNode; @@ -1378,7 +1378,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { try { synchronized (neededReplications) { for (int priority = 0; priority < blocksToReplicate.size(); priority++) { - for (Block block : blocksToReplicate.get(priority)) { + for (BlockInfo block : blocksToReplicate.get(priority)) { // block should belong to a file bc = blocksMap.getBlockCollection(block); // abandoned block or block reopened for append @@ -1462,7 +1462,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { } synchronized (neededReplications) { - Block block = rw.block; + BlockInfo block = rw.block; int priority = rw.priority; // Recheck since global lock was released // block should belong to a file @@ -1724,7 +1724,7 @@ else if (node.isDecommissionInProgress()) { * and put them back into the neededReplication queue */ private void processPendingReplications() { - Block[] timedOutItems = pendingReplications.getTimedOutBlocks(); + BlockInfo[] timedOutItems = pendingReplications.getTimedOutBlocks(); if (timedOutItems != null) { namesystem.writeLock(); try { @@ -2917,13 +2917,13 @@ private MisReplicationResult processMisReplicatedBlock(BlockInfo block) { /** Set replication for the blocks. */ public void setReplication(final short oldRepl, final short newRepl, - final String src, final Block... blocks) { + final String src, final BlockInfoContiguous... blocks) { if (newRepl == oldRepl) { return; } // update needReplication priority queues - for(Block b : blocks) { + for(BlockInfoContiguous b : blocks) { updateNeededReplications(b, 0, newRepl-oldRepl); } @@ -2931,7 +2931,7 @@ public void setReplication(final short oldRepl, final short newRepl, // old replication > the new one; need to remove copies LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl + " for " + src); - for(Block b : blocks) { + for(BlockInfoContiguous b : blocks) { processOverReplicatedBlock(b, newRepl, null, null); } } else { // replication factor is increased @@ -3114,7 +3114,8 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node); assert (namesystem.hasWriteLock()); { - if (!blocksMap.removeNode(block, node)) { + BlockInfo storedBlock = getStoredBlock(block); + if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) { blockLog.debug("BLOCK* removeStoredBlock: {} has already been" + " removed from node {}", block, node); return; @@ -3128,8 +3129,8 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { // BlockCollection bc = blocksMap.getBlockCollection(block); if (bc != null) { - namesystem.decrementSafeBlockCount(block); - updateNeededReplications(block, -1, 0); + namesystem.decrementSafeBlockCount(storedBlock); + updateNeededReplications(storedBlock, -1, 0); } // @@ -3203,7 +3204,7 @@ void addBlock(DatanodeStorageInfo storageInfo, Block block, String delHint) // // Modify the blocks->datanode map and node's map. // - pendingReplications.decrement(block, node); + pendingReplications.decrement(getStoredBlock(block), node); processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED, delHintNode); } @@ -3318,7 +3319,7 @@ public void processIncrementalBlockReport(final DatanodeID nodeID, * For a striped block, this includes nodes storing blocks belonging to the * striped block group. */ - public NumberReplicas countNodes(Block b) { + public NumberReplicas countNodes(BlockInfo b) { int decommissioned = 0; int decommissioning = 0; int live = 0; @@ -3351,11 +3352,11 @@ public NumberReplicas countNodes(Block b) { } /** - * Simpler, faster form of {@link #countNodes(Block)} that only returns the number + * Simpler, faster form of {@link #countNodes} that only returns the number * of live nodes. If in startup safemode (or its 30-sec extension period), * then it gains speed by ignoring issues of excess replicas or nodes * that are decommissioned or in process of becoming decommissioned. - * If not in startup, then it calls {@link #countNodes(Block)} instead. + * If not in startup, then it calls {@link #countNodes} instead. * * @param b - the block being tested * @return count of live nodes for this block @@ -3385,10 +3386,10 @@ void processOverReplicatedBlocksOnReCommission( if (!namesystem.isPopulatingReplQueues()) { return; } - final Iterator it = srcNode.getBlockIterator(); + final Iterator it = srcNode.getBlockIterator(); int numOverReplicated = 0; while(it.hasNext()) { - final Block block = it.next(); + final BlockInfo block = it.next(); BlockCollection bc = blocksMap.getBlockCollection(block); short expectedReplication = bc.getPreferredBlockReplication(); NumberReplicas num = countNodes(block); @@ -3452,7 +3453,7 @@ public int getTotalBlocks() { return blocksMap.size(); } - public void removeBlock(Block block) { + public void removeBlock(BlockInfo block) { assert namesystem.hasWriteLock(); // No need to ACK blocks that are being removed entirely // from the namespace, since the removal of the associated @@ -3481,7 +3482,7 @@ public BlockInfo getStoredBlock(Block block) { } /** updates a block in under replication queue */ - private void updateNeededReplications(final Block block, + private void updateNeededReplications(final BlockInfo block, final int curReplicasDelta, int expectedReplicasDelta) { namesystem.writeLock(); try { @@ -3513,7 +3514,7 @@ private void updateNeededReplications(final Block block, */ public void checkReplication(BlockCollection bc) { final short expected = bc.getPreferredBlockReplication(); - for (Block block : bc.getBlocks()) { + for (BlockInfo block : bc.getBlocks()) { final NumberReplicas n = countNodes(block); if (isNeededReplication(block, expected, n.liveReplicas())) { neededReplications.add(block, n.liveReplicas(), @@ -3690,7 +3691,7 @@ public int getCapacity() { /** * Return an iterator over the set of blocks for which there are no replicas. */ - public Iterator getCorruptReplicaBlockIterator() { + public Iterator getCorruptReplicaBlockIterator() { return neededReplications.iterator( UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS); } @@ -3815,7 +3816,7 @@ public static LocatedBlock newLocatedBlock( private static class ReplicationWork { - private final Block block; + private final BlockInfo block; private final BlockCollection bc; private final DatanodeDescriptor srcNode; @@ -3826,7 +3827,7 @@ private static class ReplicationWork { private DatanodeStorageInfo targets[]; private final int priority; - public ReplicationWork(Block block, + public ReplicationWork(BlockInfo block, BlockCollection bc, DatanodeDescriptor srcNode, List containingNodes, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java index 796b878c92d41..04232cf95e3b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java @@ -23,6 +23,7 @@ import java.sql.Time; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -46,8 +47,8 @@ class PendingReplicationBlocks { private static final Logger LOG = BlockManager.LOG; - private final Map pendingReplications; - private final ArrayList timedOutItems; + private final Map pendingReplications; + private final ArrayList timedOutItems; Daemon timerThread = null; private volatile boolean fsRunning = true; @@ -62,8 +63,8 @@ class PendingReplicationBlocks { if ( timeoutPeriod > 0 ) { this.timeout = timeoutPeriod; } - pendingReplications = new HashMap(); - timedOutItems = new ArrayList(); + pendingReplications = new HashMap<>(); + timedOutItems = new ArrayList<>(); } void start() { @@ -76,7 +77,7 @@ void start() { * @param block The corresponding block * @param targets The DataNodes where replicas of the block should be placed */ - void increment(Block block, DatanodeDescriptor[] targets) { + void increment(BlockInfo block, DatanodeDescriptor[] targets) { synchronized (pendingReplications) { PendingBlockInfo found = pendingReplications.get(block); if (found == null) { @@ -93,9 +94,9 @@ void increment(Block block, DatanodeDescriptor[] targets) { * Decrement the number of pending replication requests * for this block. * - * @param The DataNode that finishes the replication + * @param dn The DataNode that finishes the replication */ - void decrement(Block block, DatanodeDescriptor dn) { + void decrement(BlockInfo block, DatanodeDescriptor dn) { synchronized (pendingReplications) { PendingBlockInfo found = pendingReplications.get(block); if (found != null) { @@ -115,7 +116,7 @@ void decrement(Block block, DatanodeDescriptor dn) { * @param block The given block whose pending replication requests need to be * removed */ - void remove(Block block) { + void remove(BlockInfo block) { synchronized (pendingReplications) { pendingReplications.remove(block); } @@ -138,7 +139,7 @@ int size() { /** * How many copies of this block is pending replication? */ - int getNumReplicas(Block block) { + int getNumReplicas(BlockInfo block) { synchronized (pendingReplications) { PendingBlockInfo found = pendingReplications.get(block); if (found != null) { @@ -153,13 +154,13 @@ int getNumReplicas(Block block) { * replication requests. Returns null if no blocks have * timed out. */ - Block[] getTimedOutBlocks() { + BlockInfo[] getTimedOutBlocks() { synchronized (timedOutItems) { if (timedOutItems.size() <= 0) { return null; } - Block[] blockList = timedOutItems.toArray( - new Block[timedOutItems.size()]); + BlockInfo[] blockList = timedOutItems.toArray( + new BlockInfo[timedOutItems.size()]); timedOutItems.clear(); return blockList; } @@ -179,7 +180,7 @@ static class PendingBlockInfo { PendingBlockInfo(DatanodeDescriptor[] targets) { this.timeStamp = monotonicNow(); this.targets = targets == null ? new ArrayList() - : new ArrayList(Arrays.asList(targets)); + : new ArrayList<>(Arrays.asList(targets)); } long getTimeStamp() { @@ -192,9 +193,7 @@ void setTimeStamp() { void incrementReplicas(DatanodeDescriptor... newTargets) { if (newTargets != null) { - for (DatanodeDescriptor dn : newTargets) { - targets.add(dn); - } + Collections.addAll(targets, newTargets); } } @@ -232,17 +231,17 @@ public void run() { */ void pendingReplicationCheck() { synchronized (pendingReplications) { - Iterator> iter = + Iterator> iter = pendingReplications.entrySet().iterator(); long now = monotonicNow(); if(LOG.isDebugEnabled()) { LOG.debug("PendingReplicationMonitor checking Q"); } while (iter.hasNext()) { - Map.Entry entry = iter.next(); + Map.Entry entry = iter.next(); PendingBlockInfo pendingBlock = entry.getValue(); if (now > pendingBlock.getTimeStamp() + timeout) { - Block block = entry.getKey(); + BlockInfo block = entry.getKey(); synchronized (timedOutItems) { timedOutItems.add(block); } @@ -275,16 +274,14 @@ void metaSave(PrintWriter out) { synchronized (pendingReplications) { out.println("Metasave: Blocks being replicated: " + pendingReplications.size()); - Iterator> iter = - pendingReplications.entrySet().iterator(); - while (iter.hasNext()) { - Map.Entry entry = iter.next(); + for (Map.Entry entry : + pendingReplications.entrySet()) { PendingBlockInfo pendingBlock = entry.getValue(); Block block = entry.getKey(); - out.println(block + - " StartTime: " + new Time(pendingBlock.timeStamp) + - " NumReplicaInProgress: " + - pendingBlock.getNumReplicas()); + out.println(block + + " StartTime: " + new Time(pendingBlock.timeStamp) + + " NumReplicaInProgress: " + + pendingBlock.getNumReplicas()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java index 1daa0ee2bc72f..f9bce268c4ff1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -35,7 +34,7 @@ * *

* The policy for choosing which priority to give added blocks - * is implemented in {@link #getPriority(Block, int, int, int)}. + * is implemented in {@link #getPriority(int, int, int)}. *

*

The queue order is as follows:

*
    @@ -62,7 +61,7 @@ * blocks that are not corrupt higher priority. *
*/ -class UnderReplicatedBlocks implements Iterable { +class UnderReplicatedBlocks implements Iterable { /** The total number of queues : {@value} */ static final int LEVEL = 5; /** The queue with the highest priority: {@value} */ @@ -78,8 +77,8 @@ class UnderReplicatedBlocks implements Iterable { /** The queue for corrupt blocks: {@value} */ static final int QUEUE_WITH_CORRUPT_BLOCKS = 4; /** the queues themselves */ - private final List> priorityQueues - = new ArrayList>(LEVEL); + private final List> priorityQueues + = new ArrayList<>(LEVEL); /** The number of corrupt blocks with replication factor 1 */ private int corruptReplOneBlocks = 0; @@ -87,7 +86,7 @@ class UnderReplicatedBlocks implements Iterable { /** Create an object. */ UnderReplicatedBlocks() { for (int i = 0; i < LEVEL; i++) { - priorityQueues.add(new LightWeightLinkedSet()); + priorityQueues.add(new LightWeightLinkedSet()); } } @@ -131,8 +130,8 @@ synchronized int getCorruptReplOneBlockSize() { } /** Check if a block is in the neededReplication queue */ - synchronized boolean contains(Block block) { - for(LightWeightLinkedSet set : priorityQueues) { + synchronized boolean contains(BlockInfo block) { + for(LightWeightLinkedSet set : priorityQueues) { if (set.contains(block)) { return true; } @@ -141,13 +140,11 @@ synchronized boolean contains(Block block) { } /** Return the priority of a block - * @param block a under replicated block * @param curReplicas current number of replicas of the block * @param expectedReplicas expected number of replicas of the block * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1) */ - private int getPriority(Block block, - int curReplicas, + private int getPriority(int curReplicas, int decommissionedReplicas, int expectedReplicas) { assert curReplicas >= 0 : "Negative replicas!"; @@ -183,12 +180,12 @@ private int getPriority(Block block, * @param expectedReplicas expected number of replicas of the block * @return true if the block was added to a queue. */ - synchronized boolean add(Block block, + synchronized boolean add(BlockInfo block, int curReplicas, int decomissionedReplicas, int expectedReplicas) { assert curReplicas >= 0 : "Negative replicas!"; - int priLevel = getPriority(block, curReplicas, decomissionedReplicas, + int priLevel = getPriority(curReplicas, decomissionedReplicas, expectedReplicas); if(priorityQueues.get(priLevel).add(block)) { if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS && @@ -207,11 +204,11 @@ synchronized boolean add(Block block, } /** remove a block from a under replication queue */ - synchronized boolean remove(Block block, + synchronized boolean remove(BlockInfo block, int oldReplicas, int decommissionedReplicas, int oldExpectedReplicas) { - int priLevel = getPriority(block, oldReplicas, + int priLevel = getPriority(oldReplicas, decommissionedReplicas, oldExpectedReplicas); boolean removedBlock = remove(block, priLevel); @@ -241,7 +238,7 @@ synchronized boolean remove(Block block, * @param priLevel expected privilege level * @return true if the block was found and removed from one of the priority queues */ - boolean remove(Block block, int priLevel) { + boolean remove(BlockInfo block, int priLevel) { if(priLevel >= 0 && priLevel < LEVEL && priorityQueues.get(priLevel).remove(block)) { NameNode.blockStateChangeLog.debug( @@ -279,14 +276,14 @@ boolean remove(Block block, int priLevel) { * @param curReplicasDelta the change in the replicate count from before * @param expectedReplicasDelta the change in the expected replica count from before */ - synchronized void update(Block block, int curReplicas, + synchronized void update(BlockInfo block, int curReplicas, int decommissionedReplicas, int curExpectedReplicas, int curReplicasDelta, int expectedReplicasDelta) { int oldReplicas = curReplicas-curReplicasDelta; int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta; - int curPri = getPriority(block, curReplicas, decommissionedReplicas, curExpectedReplicas); - int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas); + int curPri = getPriority(curReplicas, decommissionedReplicas, curExpectedReplicas); + int oldPri = getPriority(oldReplicas, decommissionedReplicas, oldExpectedReplicas); if(NameNode.stateChangeLog.isDebugEnabled()) { NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + block + @@ -336,12 +333,12 @@ synchronized void update(Block block, int curReplicas, * @return Return a list of block lists to be replicated. The block list index * represents its replication priority. */ - public synchronized List> chooseUnderReplicatedBlocks( + public synchronized List> chooseUnderReplicatedBlocks( int blocksToProcess) { // initialize data structure for the return value - List> blocksToReplicate = new ArrayList>(LEVEL); + List> blocksToReplicate = new ArrayList<>(LEVEL); for (int i = 0; i < LEVEL; i++) { - blocksToReplicate.add(new ArrayList()); + blocksToReplicate.add(new ArrayList()); } if (size() == 0) { // There are no blocks to collect. @@ -364,7 +361,7 @@ public synchronized List> chooseUnderReplicatedBlocks( // Loop through all remaining blocks in the list. while (blockCount < blocksToProcess && neededReplicationsIterator.hasNext()) { - Block block = neededReplicationsIterator.next(); + BlockInfo block = neededReplicationsIterator.next(); blocksToReplicate.get(priority).add(block); blockCount++; } @@ -396,10 +393,10 @@ public synchronized BlockIterator iterator() { /** * An iterator over blocks. */ - class BlockIterator implements Iterator { + class BlockIterator implements Iterator { private int level; private boolean isIteratorForLevel = false; - private final List> iterators = new ArrayList>(); + private final List> iterators = new ArrayList<>(); /** * Construct an iterator over all queues. @@ -431,7 +428,7 @@ private void update() { } @Override - public Block next() { + public BlockInfo next() { if (isIteratorForLevel) { return iterators.get(0).next(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index 879738d2f7977..9abb9fa82a4e3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.util.EnumCounters; @@ -148,8 +150,8 @@ static boolean setReplication( } final short[] blockRepls = new short[2]; // 0: old, 1: new - final Block[] blocks = unprotectedSetReplication(fsd, src, replication, - blockRepls); + final BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src, + replication, blockRepls); isFile = blocks != null; if (isFile) { fsd.getEditLog().logSetReplication(src, replication); @@ -375,7 +377,7 @@ static INodeDirectory unprotectedSetQuota( } } - static Block[] unprotectedSetReplication( + static BlockInfoContiguous[] unprotectedSetReplication( FSDirectory fsd, String src, short replication, short[] blockRepls) throws QuotaExceededException, UnresolvedLinkException, SnapshotAccessControlException { @@ -410,7 +412,7 @@ static Block[] unprotectedSetReplication( blockRepls[0] = oldBR; blockRepls[1] = newBR; } - return file.getBlocks(); + return file.getContiguousBlocks(); } static void unprotectedSetStoragePolicy( 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 b9ce89e73553b..29215251221cd 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 @@ -3211,8 +3211,8 @@ FSPermissionChecker getPermissionChecker() * of blocks that need to be removed from blocksMap */ void removeBlocks(BlocksMapUpdateInfo blocks) { - List toDeleteList = blocks.getToDeleteList(); - Iterator iter = toDeleteList.iterator(); + List toDeleteList = blocks.getToDeleteList(); + Iterator iter = toDeleteList.iterator(); while (iter.hasNext()) { writeLock(); try { @@ -3268,12 +3268,11 @@ void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) { boolean trackBlockCounts = isSafeModeTrackingBlocks(); int numRemovedComplete = 0, numRemovedSafe = 0; - for (Block b : blocks.getToDeleteList()) { + for (BlockInfo b : blocks.getToDeleteList()) { if (trackBlockCounts) { - BlockInfo bi = getStoredBlock(b); - if (bi.isComplete()) { + if (b.isComplete()) { numRemovedComplete++; - if (blockManager.checkMinStorage(bi, bi.numNodes())) { + if (blockManager.checkMinStorage(b, b.numNodes())) { numRemovedSafe++; } } @@ -4185,7 +4184,8 @@ private void clearCorruptLazyPersistFiles() boolean changed = false; writeLock(); try { - final Iterator it = blockManager.getCorruptReplicaBlockIterator(); + final Iterator it = + blockManager.getCorruptReplicaBlockIterator(); while (it.hasNext()) { Block b = it.next(); @@ -5126,7 +5126,7 @@ public void incrementSafeBlockCount(int replication) { } @Override - public void decrementSafeBlockCount(Block b) { + public void decrementSafeBlockCount(BlockInfo b) { // safeMode is volatile, and may be set to null at any time SafeModeInfo safeMode = this.safeMode; if (safeMode == null) // mostly true @@ -5956,7 +5956,8 @@ Collection listCorruptFileBlocks(String path, } // print a limited # of corrupt files per call - final Iterator blkIterator = blockManager.getCorruptReplicaBlockIterator(); + final Iterator blkIterator = + blockManager.getCorruptReplicaBlockIterator(); int skip = getIntCookie(cookieTab[0]); for (int i = 0; i < skip && blkIterator.hasNext(); i++) { @@ -5964,7 +5965,7 @@ Collection listCorruptFileBlocks(String path, } while (blkIterator.hasNext()) { - Block blk = blkIterator.next(); + BlockInfo blk = blkIterator.next(); final INode inode = (INode)blockManager.getBlockCollection(blk); skip++; if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java index cf38fa50c87d8..8e05e166ed894 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java @@ -34,9 +34,9 @@ import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.DFSUtil; -import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference; import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName; @@ -950,7 +950,7 @@ public static class BlocksMapUpdateInfo { /** * The list of blocks that need to be removed from blocksMap */ - private final List toDeleteList; + private final List toDeleteList; public BlocksMapUpdateInfo() { toDeleteList = new ChunkedArrayList<>(); @@ -959,7 +959,7 @@ public BlocksMapUpdateInfo() { /** * @return The list of blocks that need to be removed from blocksMap */ - public List getToDeleteList() { + public List getToDeleteList() { return toDeleteList; } @@ -968,12 +968,12 @@ public List getToDeleteList() { * {@link BlocksMapUpdateInfo#toDeleteList} * @param toDelete the to-be-deleted block */ - public void addDeleteBlock(Block toDelete) { + public void addDeleteBlock(BlockInfo toDelete) { assert toDelete != null : "toDelete is null"; toDeleteList.add(toDelete); } - public void removeDeleteBlock(Block block) { + public void removeDeleteBlock(BlockInfo block) { assert block != null : "block is null"; toDeleteList.remove(block); } 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 b2c4ae8ef4330..88ba2cd90ef3a 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 @@ -1086,8 +1086,8 @@ void excludeSnapshotBlocks(int snapshotId, getDiffs().findEarlierSnapshotBlocks(snapshotId); if(snapshotBlocks == null) return; - List toDelete = collectedBlocks.getToDeleteList(); - for(Block blk : snapshotBlocks) { + List toDelete = collectedBlocks.getToDeleteList(); + for(BlockInfo blk : snapshotBlocks) { if(toDelete.contains(blk)) collectedBlocks.removeDeleteBlock(blk); } 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 e47e22e509a8a..24a38e5e16fbf 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 @@ -110,7 +110,7 @@ @InterfaceAudience.Private public class NamenodeFsck implements DataEncryptionKeyFactory { public static final Log LOG = LogFactory.getLog(NameNode.class.getName()); - + // return string marking fsck status public static final String CORRUPT_STATUS = "is CORRUPT"; public static final String HEALTHY_STATUS = "is HEALTHY"; @@ -118,7 +118,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED"; public static final String NONEXISTENT_STATUS = "does not exist"; public static final String FAILURE_STATUS = "FAILED"; - + private final NameNode namenode; private final NetworkTopology networktopology; private final int totalDatanodes; @@ -144,14 +144,14 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { */ private boolean internalError = false; - /** + /** * True if the user specified the -move option. * * Whe this option is in effect, we will copy salvaged blocks into the lost * and found. */ private boolean doMove = false; - /** + /** * True if the user specified the -delete option. * * Whe this option is in effect, we will delete corrupted files. @@ -184,7 +184,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { * @param remoteAddress source address of the fsck request */ NamenodeFsck(Configuration conf, NameNode namenode, - NetworkTopology networktopology, + NetworkTopology networktopology, Map pmap, PrintWriter out, int totalDatanodes, InetAddress remoteAddress) { this.conf = conf; @@ -200,7 +200,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { this.staleInterval = conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY, DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT); - + for (Iterator it = pmap.keySet().iterator(); it.hasNext();) { String key = it.next(); if (key.equals("path")) { this.path = pmap.get("path")[0]; } @@ -252,7 +252,7 @@ public void blockIdCK(String blockId) { } BlockCollection bc = bm.getBlockCollection(blockInfo); INode iNode = (INode) bc; - NumberReplicas numberReplicas= bm.countNodes(block); + NumberReplicas numberReplicas= bm.countNodes(blockInfo); out.println("Block Id: " + blockId); out.println("Block belongs to: "+iNode.getFullPathName()); out.println("No. of Expected Replica: " + @@ -351,7 +351,7 @@ public void fsck() { listCorruptFileBlocks(); return; } - + if (this.showStoragePolcies) { storageTypeSummary = new StoragePolicySummary( namenode.getNamesystem().getBlockManager().getStoragePolicies()); @@ -381,7 +381,7 @@ public void fsck() { // DFSck client scans for the string HEALTHY/CORRUPT to check the status // of file system and return appropriate code. Changing the output - // string might break testcases. Also note this must be the last line + // string might break testcases. Also note this must be the last line // of the report. if (res.isHealthy()) { out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS); @@ -424,7 +424,7 @@ private void listCorruptFileBlocks() throws IOException { + " CORRUPT files"); out.println(); } - + @VisibleForTesting void check(String parent, HdfsFileStatus file, Result res) throws IOException { String path = file.getFullName(parent); @@ -481,7 +481,7 @@ private LocatedBlocks getBlockLocations(String path, HdfsFileStatus file) throws IOException { long fileLen = file.getLen(); LocatedBlocks blocks = null; - FSNamesystem fsn = namenode.getNamesystem(); + final FSNamesystem fsn = namenode.getNamesystem(); fsn.readLock(); try { blocks = fsn.getBlockLocations( @@ -540,8 +540,10 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res ExtendedBlock block = lBlk.getBlock(); BlockManager bm = namenode.getNamesystem().getBlockManager(); + final BlockInfo storedBlock = bm.getStoredBlock( + block.getLocalBlock()); // count decommissionedReplicas / decommissioningReplicas - NumberReplicas numberReplicas = bm.countNodes(block.getLocalBlock()); + NumberReplicas numberReplicas = bm.countNodes(storedBlock); int decommissionedReplicas = numberReplicas.decommissioned();; int decommissioningReplicas = numberReplicas.decommissioning(); res.decommissionedReplicas += decommissionedReplicas; @@ -609,7 +611,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res out.println(); out.print(path + ": "); } - out.println(" Replica placement policy is violated for " + + out.println(" Replica placement policy is violated for " + block + ". " + blockPlacementStatus.getErrorDescription()); } @@ -744,7 +746,7 @@ boolean hdfsPathExists(String path) return false; } } - + private void copyBlocksToLostFound(String parent, HdfsFileStatus file, LocatedBlocks blocks) throws IOException { final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf); @@ -785,7 +787,7 @@ private void copyBlocksToLostFound(String parent, HdfsFileStatus file, fos = dfs.create(target + "/" + chain, true); chain++; } - + // copy the block. It's a pity it's not abstracted from DFSInputStream ... try { copyBlock(dfs, lblock, fos); @@ -803,7 +805,7 @@ private void copyBlocksToLostFound(String parent, HdfsFileStatus file, LOG.warn("Fsck: there were errors copying the remains of the " + "corrupted file " + fullName + " to /lost+found"); } else { - LOG.info("Fsck: copied the remains of the corrupted file " + + LOG.info("Fsck: copied the remains of the corrupted file " + fullName + " to /lost+found"); } } catch (Exception e) { @@ -814,7 +816,7 @@ private void copyBlocksToLostFound(String parent, HdfsFileStatus file, dfs.close(); } } - + /* * XXX (ab) Bulk of this method is copied verbatim from {@link DFSClient}, which is * bad. Both places should be refactored to provide a method to copy blocks @@ -825,12 +827,12 @@ private void copyBlock(final DFSClient dfs, LocatedBlock lblock, int failures = 0; InetSocketAddress targetAddr = null; TreeSet deadNodes = new TreeSet(); - BlockReader blockReader = null; - ExtendedBlock block = lblock.getBlock(); + BlockReader blockReader = null; + ExtendedBlock block = lblock.getBlock(); while (blockReader == null) { DatanodeInfo chosenNode; - + try { chosenNode = bestNode(dfs, lblock.getLocations(), deadNodes); targetAddr = NetUtils.createSocketAddr(chosenNode.getXferAddr()); @@ -901,7 +903,7 @@ public Peer newConnectedPeer(InetSocketAddress addr, bytesRead += cnt; } if ( bytesRead != block.getNumBytes() ) { - throw new IOException("Recorded block size is " + block.getNumBytes() + + throw new IOException("Recorded block size is " + block.getNumBytes() + ", but datanode returned " +bytesRead+" bytes"); } } catch (Exception e) { @@ -938,12 +940,12 @@ private DatanodeInfo bestNode(DFSClient dfs, DatanodeInfo[] nodes, } while (deadNodes.contains(chosenNode)); return chosenNode; } - + private void lostFoundInit(DFSClient dfs) { lfInited = true; try { String lfName = "/lost+found"; - + final HdfsFileStatus lfStatus = dfs.getFileInfo(lfName); if (lfStatus == null) { // not exists lfInitedOk = dfs.mkdirs(lfName, null, true); @@ -998,21 +1000,21 @@ static class Result { final short replication; final int minReplication; - + Result(Configuration conf) { - this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, + this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT); this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT); } - + /** * DFS is considered healthy if there are no missing blocks. */ boolean isHealthy() { return ((missingIds.size() == 0) && (corruptBlocks == 0)); } - + /** Add a missing block name, plus its size. */ void addMissing(String id, long size) { missingIds.add(id); @@ -1031,7 +1033,7 @@ float getReplicationFactor() { return 0.0f; return (float) (totalReplicas) / (float) totalBlocks; } - + @Override public String toString() { StringBuilder res = new StringBuilder(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java index 95fc06bd8ce5e..0debb1f77223a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java @@ -19,6 +19,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; /** SafeMode related operations. */ @InterfaceAudience.Private @@ -49,5 +50,5 @@ public interface SafeMode { public void incrementSafeBlockCount(int replication); /** Decrement number of blocks that reached minimal replication. */ - public void decrementSafeBlockCount(Block b); + public void decrementSafeBlockCount(BlockInfo b); } 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 23e610f841538..148135bae9727 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 @@ -69,9 +69,10 @@ public static int[] getReplicaInfo(final FSNamesystem namesystem, final Block b) final BlockManager bm = namesystem.getBlockManager(); namesystem.readLock(); try { + final BlockInfo storedBlock = bm.getStoredBlock(b); return new int[]{getNumberOfRacks(bm, b), - bm.countNodes(b).liveReplicas(), - bm.neededReplications.contains(b) ? 1 : 0}; + bm.countNodes(storedBlock).liveReplicas(), + bm.neededReplications.contains(storedBlock) ? 1 : 0}; } finally { namesystem.readUnlock(); } 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 4612797ab9efc..080f693628116 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 @@ -440,14 +440,14 @@ private BlockInfoContiguous addBlockOnNodes(long blockId, List list_p1 = new ArrayList(); + List list_p1 = new ArrayList<>(); list_p1.add(block); // list of lists for each priority - List> list_all = new ArrayList>(); - list_all.add(new ArrayList()); // for priority 0 + List> list_all = new ArrayList<>(); + list_all.add(new ArrayList()); // for priority 0 list_all.add(list_p1); // for priority 1 assertEquals("Block not initially pending replication", 0, 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 c3726f2a54382..1c3f075d5f426 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 @@ -166,10 +166,11 @@ void checkTimeout(String testLabel, long cycleTime) throws TimeoutException { /* threadsafe read of the replication counts for this block */ NumberReplicas countNodes(Block block, FSNamesystem namesystem) { + BlockManager blockManager = namesystem.getBlockManager(); namesystem.readLock(); try { lastBlock = block; - lastNum = namesystem.getBlockManager().countNodes(block); + lastNum = blockManager.countNodes(blockManager.getStoredBlock(block)); return lastNum; } finally { 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 a86b57347f35a..2d7bb440d0cd8 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 @@ -117,7 +117,8 @@ public void testProcesOverReplicateBlock() throws Exception { // corrupt one won't be chosen to be excess one // without 4910 the number of live replicas would be 0: block gets lost - assertEquals(1, bm.countNodes(block.getLocalBlock()).liveReplicas()); + assertEquals(1, bm.countNodes( + bm.getStoredBlock(block.getLocalBlock())).liveReplicas()); } } finally { namesystem.writeUnlock(); @@ -219,7 +220,7 @@ public void testInvalidateOverReplicatedBlock() throws Exception { out.close(); ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p); assertEquals("Expected only one live replica for the block", 1, bm - .countNodes(block.getLocalBlock()).liveReplicas()); + .countNodes(bm.getStoredBlock(block.getLocalBlock())).liveReplicas()); } finally { cluster.shutdown(); } 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 259404e8701a8..b714afa6840cc 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 @@ -53,6 +53,12 @@ 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 BlockInfo genBlockInfo(long id, long length, long gs) { + return new BlockInfoContiguous(new Block(id, length, gs), + (short) DATANODE_COUNT); + } + @Test public void testPendingReplication() { PendingReplicationBlocks pendingReplications; @@ -63,7 +69,7 @@ public void testPendingReplication() { // DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10); for (int i = 0; i < storages.length; i++) { - Block block = new Block(i, i, 0); + BlockInfo block = genBlockInfo(i, i, 0); DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i]; System.arraycopy(storages, 0, targets, 0, i); pendingReplications.increment(block, @@ -76,7 +82,7 @@ public void testPendingReplication() { // // remove one item and reinsert it // - Block blk = new Block(8, 8, 0); + BlockInfo blk = genBlockInfo(8, 8, 0); pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica assertEquals("pendingReplications.getNumReplicas ", 7, pendingReplications.getNumReplicas(blk)); @@ -96,7 +102,7 @@ public void testPendingReplication() { // are sane. // for (int i = 0; i < 10; i++) { - Block block = new Block(i, i, 0); + BlockInfo block = genBlockInfo(i, i, 0); int numReplicas = pendingReplications.getNumReplicas(block); assertTrue(numReplicas == i); } @@ -115,7 +121,7 @@ public void testPendingReplication() { } for (int i = 10; i < 15; i++) { - Block block = new Block(i, i, 0); + BlockInfo block = genBlockInfo(i, i, 0); pendingReplications.increment(block, DatanodeStorageInfo.toDatanodeDescriptors( DFSTestUtil.createDatanodeStorageInfos(i))); @@ -275,7 +281,7 @@ public void testBlockReceived() throws Exception { assertEquals(1, blkManager.pendingReplications.size()); INodeFile fileNode = fsn.getFSDirectory().getINode4Write(file).asFile(); - Block[] blocks = fileNode.getBlocks(); + BlockInfo[] blocks = fileNode.getBlocks(); assertEquals(DATANODE_COUNT - 1, blkManager.pendingReplications.getNumReplicas(blocks[0])); @@ -381,8 +387,8 @@ public void testPendingAndInvalidate() throws Exception { BlockManagerTestUtil.computeAllPendingWork(bm); BlockManagerTestUtil.updateState(bm); assertEquals(bm.getPendingReplicationBlocksCount(), 1L); - assertEquals(bm.pendingReplications.getNumReplicas(block.getBlock() - .getLocalBlock()), 2); + BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock()); + assertEquals(bm.pendingReplications.getNumReplicas(storedBlock), 2); // 4. delete the file fs.delete(filePath, true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java index 728934d65d1a5..1a32892a9432d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java @@ -58,7 +58,9 @@ public class TestRBWBlockInvalidation { private static NumberReplicas countReplicas(final FSNamesystem namesystem, ExtendedBlock block) { - return namesystem.getBlockManager().countNodes(block.getLocalBlock()); + final BlockManager blockManager = namesystem.getBlockManager(); + return blockManager.countNodes(blockManager.getStoredBlock( + block.getLocalBlock())); } /** 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 54e7108ccc5c1..9e2e1727d83e1 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 @@ -831,6 +831,10 @@ public void testRereplicate3() throws Exception { assertEquals(targets.length, 2); assertTrue(isOnSameRack(targets[0], dataNodes[2])); } + + private BlockInfo genBlockInfo(long id) { + return new BlockInfoContiguous(new Block(id), (short) 3); + } /** * Test for the high priority blocks are processed before the low priority @@ -850,16 +854,17 @@ public void testReplicationWithPriority() throws Exception { .getNamesystem().getBlockManager().neededReplications; for (int i = 0; i < 100; i++) { // Adding the blocks directly to normal priority - neededReplications.add(new Block(ThreadLocalRandom.current() - .nextLong()), 2, 0, 3); + + neededReplications.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 2, 0, 3); } // Lets wait for the replication interval, to start process normal // priority blocks Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL); // Adding the block directly to high priority list - neededReplications.add(new Block(ThreadLocalRandom.current().nextLong()), - 1, 0, 3); + neededReplications.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 1, 0, 3); // Lets wait for the replication interval Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL); @@ -882,30 +887,31 @@ public void testChooseUnderReplicatedBlocks() throws Exception { for (int i = 0; i < 5; i++) { // Adding QUEUE_HIGHEST_PRIORITY block - underReplicatedBlocks.add(new Block(ThreadLocalRandom.current() - .nextLong()), 1, 0, 3); + underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 1, 0, 3); // Adding QUEUE_VERY_UNDER_REPLICATED block - underReplicatedBlocks.add(new Block(ThreadLocalRandom.current() - .nextLong()), 2, 0, 7); + underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 2, 0, 7); // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block - underReplicatedBlocks.add(new Block(ThreadLocalRandom.current() - .nextLong()), 6, 0, 6); + underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 6, 0, 6); // Adding QUEUE_UNDER_REPLICATED block - underReplicatedBlocks.add(new Block(ThreadLocalRandom.current() - .nextLong()), 5, 0, 6); + underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 5, 0, 6); // Adding QUEUE_WITH_CORRUPT_BLOCKS block - underReplicatedBlocks.add(new Block(ThreadLocalRandom.current() - .nextLong()), 0, 0, 3); + underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 0, 0, 3); } // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks // from // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED. - List> chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(6); + List> chosenBlocks = + underReplicatedBlocks.chooseUnderReplicatedBlocks(6); assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0); // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from @@ -915,8 +921,8 @@ public void testChooseUnderReplicatedBlocks() throws Exception { assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0); // Adding QUEUE_HIGHEST_PRIORITY - underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().nextLong()), - 1, 0, 3); + underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current(). + nextLong()), 1, 0, 3); // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED @@ -934,7 +940,7 @@ public void testChooseUnderReplicatedBlocks() throws Exception { /** asserts the chosen blocks with expected priority blocks */ private void assertTheChosenBlocks( - List> chosenBlocks, int firstPrioritySize, + List> chosenBlocks, int firstPrioritySize, int secondPrioritySize, int thirdPrioritySize, int fourthPrioritySize, int fifthPrioritySize) { assertEquals( @@ -1108,9 +1114,9 @@ public void testGetReplWorkMultiplier() { public void testUpdateDoesNotCauseSkippedReplication() { UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks(); - Block block1 = new Block(ThreadLocalRandom.current().nextLong()); - Block block2 = new Block(ThreadLocalRandom.current().nextLong()); - Block block3 = new Block(ThreadLocalRandom.current().nextLong()); + BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + BlockInfo block3 = genBlockInfo(ThreadLocalRandom.current().nextLong()); // Adding QUEUE_VERY_UNDER_REPLICATED block final int block1CurReplicas = 2; @@ -1124,7 +1130,7 @@ public void testUpdateDoesNotCauseSkippedReplication() { // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block3, 2, 0, 6); - List> chosenBlocks; + List> chosenBlocks; // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block // from QUEUE_VERY_UNDER_REPLICATED. @@ -1156,9 +1162,9 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() when(mockNS.hasWriteLock()).thenReturn(true); BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; - - Block block1 = new Block(ThreadLocalRandom.current().nextLong()); - Block block2 = new Block(ThreadLocalRandom.current().nextLong()); + + BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); @@ -1166,7 +1172,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block2, 0, 1, 1); - List> chosenBlocks; + List> chosenBlocks; // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block // from QUEUE_VERY_UNDER_REPLICATED. @@ -1203,8 +1209,13 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; +<<<<<<< HEAD Block block1 = new Block(ThreadLocalRandom.current().nextLong()); Block block2 = new Block(ThreadLocalRandom.current().nextLong()); +======= + BlockInfo block1 = genBlockInfo(random.nextLong()); + BlockInfo block2 = genBlockInfo(random.nextLong()); +>>>>>>> 3e6f458... HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao. // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); @@ -1212,7 +1223,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block2, 0, 1, 1); - List> chosenBlocks; + List> chosenBlocks; // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block // from QUEUE_VERY_UNDER_REPLICATED. @@ -1266,8 +1277,13 @@ public void testupdateNeededReplicationsDoesNotCauseSkippedReplication() BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; +<<<<<<< HEAD Block block1 = new Block(ThreadLocalRandom.current().nextLong()); Block block2 = new Block(ThreadLocalRandom.current().nextLong()); +======= + BlockInfo block1 = genBlockInfo(random.nextLong()); + BlockInfo block2 = genBlockInfo(random.nextLong()); +>>>>>>> 3e6f458... HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao. // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); @@ -1275,14 +1291,14 @@ public void testupdateNeededReplicationsDoesNotCauseSkippedReplication() // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block2, 0, 1, 1); - List> chosenBlocks; + List> chosenBlocks; // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block // from QUEUE_VERY_UNDER_REPLICATED. chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1); assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0); - bm.setReplication((short)0, (short)1, "", block1); + bm.setReplication((short)0, (short)1, "", (BlockInfoContiguous) block1); // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block // from QUEUE_VERY_UNDER_REPLICATED. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java index e87a043762c28..de36e077aea81 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java @@ -28,6 +28,10 @@ public class TestUnderReplicatedBlockQueues { + private BlockInfo genBlockInfo(long id) { + return new BlockInfoContiguous(new Block(id), (short) 3); + } + /** * Test that adding blocks with different replication counts puts them * into different queues @@ -36,11 +40,11 @@ public class TestUnderReplicatedBlockQueues { @Test public void testBlockPriorities() throws Throwable { UnderReplicatedBlocks queues = new UnderReplicatedBlocks(); - Block block1 = new Block(1); - Block block2 = new Block(2); - Block block_very_under_replicated = new Block(3); - Block block_corrupt = new Block(4); - Block block_corrupt_repl_one = new Block(5); + BlockInfo block1 = genBlockInfo(1); + BlockInfo block2 = genBlockInfo(2); + BlockInfo block_very_under_replicated = genBlockInfo(3); + BlockInfo block_corrupt = genBlockInfo(4); + BlockInfo block_corrupt_repl_one = genBlockInfo(5); //add a block with a single entry assertAdded(queues, block1, 1, 0, 3); @@ -82,7 +86,7 @@ public void testBlockPriorities() throws Throwable { } private void assertAdded(UnderReplicatedBlocks queues, - Block block, + BlockInfo block, int curReplicas, int decomissionedReplicas, int expectedReplicas) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java index 8f99afba1f766..5604246e96294 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +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.DatanodeManager; @@ -81,6 +82,7 @@ public class TestReadOnlySharedStorage { private DatanodeInfo readOnlyDataNode; private Block block; + private BlockInfo storedBlock; private ExtendedBlock extendedBlock; @@ -132,6 +134,7 @@ public void setup() throws IOException, InterruptedException { LocatedBlock locatedBlock = getLocatedBlock(); extendedBlock = locatedBlock.getBlock(); block = extendedBlock.getLocalBlock(); + storedBlock = blockManager.getStoredBlock(block); assertThat(locatedBlock.getLocations().length, is(1)); normalDataNode = locatedBlock.getLocations()[0]; @@ -188,7 +191,7 @@ private void validateStorageState(StorageReport[] storageReports, DatanodeStorag } private void validateNumberReplicas(int expectedReplicas) throws IOException { - NumberReplicas numberReplicas = blockManager.countNodes(block); + NumberReplicas numberReplicas = blockManager.countNodes(storedBlock); assertThat(numberReplicas.liveReplicas(), is(expectedReplicas)); assertThat(numberReplicas.excessReplicas(), is(0)); assertThat(numberReplicas.corruptReplicas(), is(0)); @@ -230,7 +233,7 @@ public void testNormalReplicaOffline() throws Exception { cluster.getNameNode(), normalDataNode.getXferAddr()); // The live replica count should now be zero (since the NORMAL replica is offline) - NumberReplicas numberReplicas = blockManager.countNodes(block); + NumberReplicas numberReplicas = blockManager.countNodes(storedBlock); assertThat(numberReplicas.liveReplicas(), is(0)); // The block should be reported as under-replicated @@ -263,7 +266,7 @@ public void testReadOnlyReplicaCorrupt() throws Exception { waitForLocations(1); // However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count - NumberReplicas numberReplicas = blockManager.countNodes(block); + NumberReplicas numberReplicas = blockManager.countNodes(storedBlock); assertThat(numberReplicas.corruptReplicas(), is(0)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java index 37abc5b726d87..228a6672c3787 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.junit.Test; @@ -260,7 +261,9 @@ public void testWithAllCorruptReplicas() throws Exception { } private static NumberReplicas countReplicas(final FSNamesystem namesystem, ExtendedBlock block) { - return namesystem.getBlockManager().countNodes(block.getLocalBlock()); + final BlockManager blockManager = namesystem.getBlockManager(); + return blockManager.countNodes(blockManager.getStoredBlock( + block.getLocalBlock())); } private void corruptBlock(MiniDFSCluster cluster, FileSystem fs, final Path fileName, From c3bc083405d84a368bf7281bc109bac2a7c62d0c Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Wed, 18 Mar 2015 19:21:37 +0800 Subject: [PATCH 022/212] HADOOP-11706 Refine a little bit erasure coder API --- .../io/erasurecode/coder/ErasureCoder.java | 4 +++- .../erasurecode/rawcoder/RawErasureCoder.java | 4 +++- .../hadoop/io/erasurecode/TestCoderBase.java | 17 ++++++++++++--- .../coder/TestErasureCoderBase.java | 21 +++---------------- .../erasurecode/rawcoder/TestJRSRawCoder.java | 12 +++++------ .../rawcoder/TestRawCoderBase.java | 2 ++ 6 files changed, 31 insertions(+), 29 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java index 68875c06ad833..c5922f37b15f1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.io.erasurecode.coder; +import org.apache.hadoop.conf.Configurable; + /** * An erasure coder to perform encoding or decoding given a group. Generally it * involves calculating necessary internal steps according to codec logic. For @@ -31,7 +33,7 @@ * of multiple coding steps. * */ -public interface ErasureCoder { +public interface ErasureCoder extends Configurable { /** * Initialize with the important parameters for the code. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java index 91a9abfe2aad4..9af5b6c0a9459 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; +import org.apache.hadoop.conf.Configurable; + /** * RawErasureCoder is a common interface for {@link RawErasureEncoder} and * {@link RawErasureDecoder} as both encoder and decoder share some properties. @@ -31,7 +33,7 @@ * low level constructs, since it only takes care of the math calculation with * a group of byte buffers. */ -public interface RawErasureCoder { +public interface RawErasureCoder extends Configurable { /** * Initialize with the important parameters for the code. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 194413a43ac49..22fd98d2c4d2e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -17,11 +17,12 @@ */ package org.apache.hadoop.io.erasurecode; +import org.apache.hadoop.conf.Configuration; + import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Random; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertTrue; /** @@ -31,6 +32,7 @@ public abstract class TestCoderBase { protected static Random RAND = new Random(); + private Configuration conf; protected int numDataUnits; protected int numParityUnits; protected int chunkSize = 16 * 1024; @@ -49,14 +51,23 @@ public abstract class TestCoderBase { * @param numParityUnits * @param erasedIndexes */ - protected void prepare(int numDataUnits, int numParityUnits, - int[] erasedIndexes) { + protected void prepare(Configuration conf, int numDataUnits, + int numParityUnits, int[] erasedIndexes) { + this.conf = conf; this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; this.erasedDataIndexes = erasedIndexes != null ? erasedIndexes : new int[] {0}; } + /** + * Get the conf the test. + * @return configuration + */ + protected Configuration getConf() { + return this.conf; + } + /** * Compare and verify if erased chunks are equal to recovered chunks * @param erasedChunks diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index d911db9e19921..b963a594eb112 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -17,10 +17,9 @@ */ package org.apache.hadoop.io.erasurecode.coder; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.erasurecode.ECBlock; -import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.TestCoderBase; /** @@ -30,7 +29,6 @@ public abstract class TestErasureCoderBase extends TestCoderBase { protected Class encoderClass; protected Class decoderClass; - private Configuration conf; protected int numChunksInBlock = 16; /** @@ -47,19 +45,6 @@ public TestBlock(ECChunk[] chunks) { } } - /** - * Prepare before running the case. - * @param conf - * @param numDataUnits - * @param numParityUnits - * @param erasedIndexes - */ - protected void prepare(Configuration conf, int numDataUnits, - int numParityUnits, int[] erasedIndexes) { - this.conf = conf; - super.prepare(numDataUnits, numParityUnits, erasedIndexes); - } - /** * Generating source data, encoding, recovering and then verifying. * RawErasureCoder mainly uses ECChunk to pass input and output data buffers, @@ -162,7 +147,7 @@ private ErasureEncoder createEncoder() { } encoder.initialize(numDataUnits, numParityUnits, chunkSize); - ((AbstractErasureCoder)encoder).setConf(conf); + encoder.setConf(getConf()); return encoder; } @@ -179,7 +164,7 @@ private ErasureDecoder createDecoder() { } decoder.initialize(numDataUnits, numParityUnits, chunkSize); - ((AbstractErasureCoder)decoder).setConf(conf); + decoder.setConf(getConf()); return decoder; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java index e54f647cf6c8f..39e5deb452765 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java @@ -46,37 +46,37 @@ public void setup() { @Test public void testCodingNoDirectBuffer_10x4() { - prepare(10, 4, null); + prepare(null, 10, 4, null); testCoding(false); } @Test public void testCodingDirectBuffer_10x4() { - prepare(10, 4, null); + prepare(null, 10, 4, null); testCoding(true); } @Test public void testCodingDirectBuffer_10x4_erasure_of_2_4() { - prepare(10, 4, new int[] {2, 4}); + prepare(null, 10, 4, new int[] {2, 4}); testCoding(true); } @Test public void testCodingDirectBuffer_10x4_erasing_all() { - prepare(10, 4, new int[] {0, 1, 2, 3}); + prepare(null, 10, 4, new int[] {0, 1, 2, 3}); testCoding(true); } @Test public void testCodingNoDirectBuffer_3x3() { - prepare(3, 3, null); + prepare(null, 3, 3, null); testCoding(false); } @Test public void testCodingDirectBuffer_3x3() { - prepare(3, 3, null); + prepare(null, 3, 3, null); testCoding(true); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 890f632370f12..b036eed4ced15 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -86,6 +86,7 @@ protected RawErasureEncoder createEncoder() { } encoder.initialize(numDataUnits, numParityUnits, chunkSize); + encoder.setConf(getConf()); return encoder; } @@ -102,6 +103,7 @@ protected RawErasureDecoder createDecoder() { } decoder.initialize(numDataUnits, numParityUnits, chunkSize); + decoder.setConf(getConf()); return decoder; } From e3dbfeb13827b6d78cedcb5e3b1ede3a7b6aecba Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Wed, 18 Mar 2015 19:24:24 +0800 Subject: [PATCH 023/212] Updated CHANGES-HDFS-EC-7285.txt accordingly --- hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index a97dc34b87bc4..e27ff5c1f867d 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -19,6 +19,9 @@ ( Kai Zheng via vinayakumarb ) HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng - ( Kai Zheng ) + ( Kai Zheng ) + + HADOOP-11706. Refine a little bit erasure coder API. Contributed by Kai Zheng + ( Kai Zheng ) From 57a84c0d149b693c913416975cafe6de4e23c321 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 18 Mar 2015 15:52:36 -0700 Subject: [PATCH 024/212] HDFS-7369. Erasure coding: distribute recovery work for striped blocks to DataNode. Contributed by Zhe Zhang. --- .../blockmanagement/BlockCollection.java | 5 + .../server/blockmanagement/BlockManager.java | 290 +++++++++++++----- .../blockmanagement/DatanodeDescriptor.java | 72 ++++- .../blockmanagement/DatanodeManager.java | 20 +- .../hdfs/server/namenode/INodeFile.java | 8 +- .../protocol/BlockECRecoveryCommand.java | 63 ++++ .../server/protocol/DatanodeProtocol.java | 1 + .../blockmanagement/BlockManagerTestUtil.java | 2 +- .../blockmanagement/TestBlockManager.java | 22 +- .../TestRecoverStripedBlocks.java | 107 +++++++ .../TestReplicationPolicy.java | 2 +- 11 files changed, 484 insertions(+), 108 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java index 250bdcd5ecf16..928424ba20463 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java @@ -86,4 +86,9 @@ public void convertLastBlockToUC(BlockInfo lastBlock, * @return whether the block collection is under construction. */ public boolean isUnderConstruction(); + + /** + * @return whether the block collection is in striping format + */ + public boolean isStriped(); } 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 7d4139c7376bc..bbead63f31317 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 @@ -43,6 +43,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; @@ -534,9 +535,9 @@ private void dumpBlockMeta(Block block, PrintWriter out) { NumberReplicas numReplicas = new NumberReplicas(); // source node returned is not used - chooseSourceDatanode(block, containingNodes, + chooseSourceDatanodes(getStoredBlock(block), containingNodes, containingLiveReplicasNodes, numReplicas, - UnderReplicatedBlocks.LEVEL); + new LinkedList(), 1, UnderReplicatedBlocks.LEVEL); // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are // not included in the numReplicas.liveReplicas() count @@ -1337,15 +1338,15 @@ int computeInvalidateWork(int nodesToProcess) { } /** - * Scan blocks in {@link #neededReplications} and assign replication - * work to data-nodes they belong to. + * Scan blocks in {@link #neededReplications} and assign recovery + * (replication or erasure coding) work to data-nodes they belong to. * * The number of process blocks equals either twice the number of live * data-nodes or the number of under-replicated blocks whichever is less. * * @return number of blocks scheduled for replication during this iteration. */ - int computeReplicationWork(int blocksToProcess) { + int computeBlockRecoveryWork(int blocksToProcess) { List> blocksToReplicate = null; namesystem.writeLock(); try { @@ -1355,30 +1356,32 @@ int computeReplicationWork(int blocksToProcess) { } finally { namesystem.writeUnlock(); } - return computeReplicationWorkForBlocks(blocksToReplicate); + return computeRecoveryWorkForBlocks(blocksToReplicate); } - /** Replicate a set of blocks + /** + * Recover a set of blocks to full strength through replication or + * erasure coding * - * @param blocksToReplicate blocks to be replicated, for each priority + * @param blocksToRecover blocks to be recovered, for each priority * @return the number of blocks scheduled for replication */ @VisibleForTesting - int computeReplicationWorkForBlocks(List> blocksToReplicate) { + int computeRecoveryWorkForBlocks(List> blocksToRecover) { int requiredReplication, numEffectiveReplicas; List containingNodes; - DatanodeDescriptor srcNode; BlockCollection bc = null; int additionalReplRequired; int scheduledWork = 0; - List work = new LinkedList(); + List recovWork = new LinkedList<>(); + // Step 1: categorize at-risk blocks into replication and EC tasks namesystem.writeLock(); try { synchronized (neededReplications) { - for (int priority = 0; priority < blocksToReplicate.size(); priority++) { - for (BlockInfo block : blocksToReplicate.get(priority)) { + for (int priority = 0; priority < blocksToRecover.size(); priority++) { + for (BlockInfo block : blocksToRecover.get(priority)) { // block should belong to a file bc = blocksMap.getBlockCollection(block); // abandoned block or block reopened for append @@ -1392,25 +1395,31 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { requiredReplication = bc.getPreferredBlockReplication(); // get a source data-node - containingNodes = new ArrayList(); - List liveReplicaNodes = new ArrayList(); + containingNodes = new ArrayList<>(); + List liveReplicaNodes = new ArrayList<>(); NumberReplicas numReplicas = new NumberReplicas(); - srcNode = chooseSourceDatanode( + List missingBlockIndices = new LinkedList<>(); + DatanodeDescriptor[] srcNodes; + int numSourceNodes = bc.isStriped() ? + HdfsConstants.NUM_DATA_BLOCKS : 1; + srcNodes = chooseSourceDatanodes( block, containingNodes, liveReplicaNodes, numReplicas, - priority); - if(srcNode == null) { // block can not be replicated from any node - LOG.debug("Block " + block + " cannot be repl from any node"); + missingBlockIndices, numSourceNodes, priority); + if(srcNodes == null || srcNodes.length == 0) { + // block can not be replicated from any node + LOG.debug("Block " + block + " cannot be recovered " + + "from any node"); continue; } - // liveReplicaNodes can include READ_ONLY_SHARED replicas which are + // liveReplicaNodes can include READ_ONLY_SHARED replicas which are // not included in the numReplicas.liveReplicas() count assert liveReplicaNodes.size() >= numReplicas.liveReplicas(); // do not schedule more if enough replicas is already pending numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplications.getNumReplicas(block); - + if (numEffectiveReplicas >= requiredReplication) { if ( (pendingReplications.getNumReplicas(block) > 0) || (blockHasEnoughRacks(block)) ) { @@ -1427,9 +1436,21 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { } else { additionalReplRequired = 1; // Needed on a new rack } - work.add(new ReplicationWork(block, bc, srcNode, - containingNodes, liveReplicaNodes, additionalReplRequired, - priority)); + if (bc.isStriped()) { + ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes, + containingNodes, liveReplicaNodes, additionalReplRequired, + priority); + short[] missingBlockArray = new short[missingBlockIndices.size()]; + for (int i = 0 ; i < missingBlockIndices.size(); i++) { + missingBlockArray[i] = missingBlockIndices.get(i); + } + ecw.setMissingBlockIndices(missingBlockArray); + recovWork.add(ecw); + } else { + recovWork.add(new ReplicationWork(block, bc, srcNodes, + containingNodes, liveReplicaNodes, additionalReplRequired, + priority)); + } } } } @@ -1437,8 +1458,9 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { namesystem.writeUnlock(); } + // Step 2: choose target nodes for each recovery task final Set excludedNodes = new HashSet(); - for(ReplicationWork rw : work){ + for(BlockRecoveryWork rw : recovWork){ // Exclude all of the containing nodes from being targets. // This list includes decommissioning or corrupt nodes. excludedNodes.clear(); @@ -1452,9 +1474,10 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes); } + // Step 3: add tasks to the DN namesystem.writeLock(); try { - for(ReplicationWork rw : work){ + for(BlockRecoveryWork rw : recovWork){ final DatanodeStorageInfo[] targets = rw.targets; if(targets == null || targets.length == 0){ rw.targets = null; @@ -1493,7 +1516,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { if ( (numReplicas.liveReplicas() >= requiredReplication) && (!blockHasEnoughRacks(block)) ) { - if (rw.srcNode.getNetworkLocation().equals( + if (rw.srcNodes[0].getNetworkLocation().equals( targets[0].getDatanodeDescriptor().getNetworkLocation())) { //No use continuing, unless a new rack in this case continue; @@ -1501,7 +1524,17 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { } // Add block to the to be replicated list - rw.srcNode.addBlockToBeReplicated(block, targets); + if (bc.isStriped()) { + assert rw instanceof ErasureCodingWork; + assert rw.targets.length > 0; + rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded( + new ExtendedBlock(namesystem.getBlockPoolId(), block), + rw.srcNodes, rw.targets, + ((ErasureCodingWork)rw).getMissingBlockIndicies()); + } + else { + rw.srcNodes[0].addBlockToBeReplicated(block, targets); + } scheduledWork++; DatanodeStorageInfo.incrementBlocksScheduled(targets); @@ -1525,7 +1558,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { if (blockLog.isInfoEnabled()) { // log which blocks have been scheduled for replication - for(ReplicationWork rw : work){ + for(BlockRecoveryWork rw : recovWork){ DatanodeStorageInfo[] targets = rw.targets; if (targets != null && targets.length != 0) { StringBuilder targetList = new StringBuilder("datanode(s)"); @@ -1533,7 +1566,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { targetList.append(' '); targetList.append(targets[k].getDatanodeDescriptor()); } - blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNode, + blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes, rw.block, targetList); } } @@ -1619,55 +1652,66 @@ List getDatanodeDescriptors(List nodes) { } /** - * Parse the data-nodes the block belongs to and choose one, - * which will be the replication source. + * Parse the data-nodes the block belongs to and choose a certain number + * from them to be the recovery sources. * * We prefer nodes that are in DECOMMISSION_INPROGRESS state to other nodes * since the former do not have write traffic and hence are less busy. * We do not use already decommissioned nodes as a source. - * Otherwise we choose a random node among those that did not reach their - * replication limits. However, if the replication is of the highest priority - * and all nodes have reached their replication limits, we will choose a - * random node despite the replication limit. + * Otherwise we randomly choose nodes among those that did not reach their + * replication limits. However, if the recovery work is of the highest + * priority and all nodes have reached their replication limits, we will + * randomly choose the desired number of nodes despite the replication limit. * * In addition form a list of all nodes containing the block * and calculate its replication numbers. * * @param block Block for which a replication source is needed - * @param containingNodes List to be populated with nodes found to contain the - * given block - * @param nodesContainingLiveReplicas List to be populated with nodes found to - * contain live replicas of the given block - * @param numReplicas NumberReplicas instance to be initialized with the - * counts of live, corrupt, excess, and - * decommissioned replicas of the given - * block. + * @param containingNodes List to be populated with nodes found to contain + * the given block + * @param nodesContainingLiveReplicas List to be populated with nodes found + * to contain live replicas of the given + * block + * @param numReplicas NumberReplicas instance to be initialized with the + * counts of live, corrupt, excess, and decommissioned + * replicas of the given block. + * @param missingBlockIndices List to be populated with indices of missing + * blocks in a striped block group or missing + * replicas of a replicated block + * @param numSourceNodes integer specifying the number of source nodes to + * choose * @param priority integer representing replication priority of the given * block - * @return the DatanodeDescriptor of the chosen node from which to replicate - * the given block - */ - @VisibleForTesting - DatanodeDescriptor chooseSourceDatanode(Block block, - List containingNodes, - List nodesContainingLiveReplicas, - NumberReplicas numReplicas, - int priority) { + * @return the array of DatanodeDescriptor of the chosen nodes from which to + * recover the given block + */ + @VisibleForTesting + DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block, + List containingNodes, + List nodesContainingLiveReplicas, + NumberReplicas numReplicas, + List missingBlockIndices, int numSourceNodes, int priority) { containingNodes.clear(); nodesContainingLiveReplicas.clear(); - DatanodeDescriptor srcNode = null; + LinkedList srcNodes = new LinkedList<>(); int live = 0; int decommissioned = 0; int decommissioning = 0; int corrupt = 0; int excess = 0; - + missingBlockIndices.clear(); + Set healthyIndices = new HashSet<>(); + Collection nodesCorrupt = corruptReplicas.getNodes(block); for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + if (block.isStriped()) { + healthyIndices.add((short) ((BlockInfoStriped) block). + getStorageBlockIndex(storage)); + } final DatanodeDescriptor node = storage.getDatanodeDescriptor(); LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node.getDatanodeUuid()); - int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; + int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) corrupt += countableReplica; else if (node.isDecommissionInProgress()) { @@ -1703,20 +1747,32 @@ else if (node.isDecommissionInProgress()) { continue; // We got this far, current node is a reasonable choice - if (srcNode == null) { - srcNode = node; + if(srcNodes.size() < numSourceNodes) { + srcNodes.add(node); continue; } // switch to a different node randomly // this to prevent from deterministically selecting the same node even // if the node failed to replicate the block on previous iterations - if(ThreadLocalRandom.current().nextBoolean()) - srcNode = node; + if(ThreadLocalRandom.current().nextBoolean()) { + int pos = ThreadLocalRandom.current().nextInt(numSourceNodes); + if(!srcNodes.get(pos).isDecommissionInProgress()) { + srcNodes.set(pos, node); + } + } + } + if (block.isStriped()) { + for (short i = 0; i < HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; i++) { + if (!healthyIndices.contains(i)) { + missingBlockIndices.add(i); + } + } } if(numReplicas != null) numReplicas.initialize(live, decommissioned, decommissioning, corrupt, excess, 0); - return srcNode; + return srcNodes.toArray(new DatanodeDescriptor[srcNodes.size()]); } /** @@ -1751,7 +1807,7 @@ private void processPendingReplications() { */ } } - + /** * StatefulBlockInfo is used to build the "toUC" list, which is a list of * updates to the information about under-construction blocks. @@ -3716,7 +3772,7 @@ public int numOfUnderReplicatedBlocks() { } /** - * Periodically calls computeReplicationWork(). + * Periodically calls computeBlockRecoveryWork(). */ private class ReplicationMonitor implements Runnable { @@ -3774,7 +3830,7 @@ int computeDatanodeWork() { final int nodesToProcess = (int) Math.ceil(numlive * this.blocksInvalidateWorkPct); - int workFound = this.computeReplicationWork(blocksToProcess); + int workFound = this.computeBlockRecoveryWork(blocksToProcess); // Update counters namesystem.writeLock(); @@ -3814,47 +3870,117 @@ public static LocatedBlock newLocatedBlock( null); } - private static class ReplicationWork { - - private final BlockInfo block; - private final BlockCollection bc; + /** + * This class is used internally by {@link this#computeRecoveryWorkForBlocks} + * to represent a task to recover a block through replication or erasure + * coding. Recovery is done by transferring data from {@link srcNodes} to + * {@link targets} + */ + private static class BlockRecoveryWork { + protected final BlockInfo block; + protected final BlockCollection bc; - private final DatanodeDescriptor srcNode; - private final List containingNodes; - private final List liveReplicaStorages; - private final int additionalReplRequired; + /** + * An erasure coding recovery task has multiple source nodes. + * A replication task only has 1 source node, stored on top of the array + */ + protected final DatanodeDescriptor[] srcNodes; + /** Nodes containing the block; avoid them in choosing new targets */ + protected final List containingNodes; + /** Required by {@link BlockPlacementPolicy#chooseTarget} */ + protected final List liveReplicaStorages; + protected final int additionalReplRequired; - private DatanodeStorageInfo targets[]; - private final int priority; + protected DatanodeStorageInfo[] targets; + protected final int priority; - public ReplicationWork(BlockInfo block, + public BlockRecoveryWork(BlockInfo block, BlockCollection bc, - DatanodeDescriptor srcNode, + DatanodeDescriptor[] srcNodes, List containingNodes, List liveReplicaStorages, int additionalReplRequired, int priority) { this.block = block; this.bc = bc; - this.srcNode = srcNode; - this.srcNode.incrementPendingReplicationWithoutTargets(); + this.srcNodes = srcNodes; this.containingNodes = containingNodes; this.liveReplicaStorages = liveReplicaStorages; this.additionalReplRequired = additionalReplRequired; this.priority = priority; this.targets = null; } - - private void chooseTargets(BlockPlacementPolicy blockplacement, + + protected void chooseTargets(BlockPlacementPolicy blockplacement, + BlockStoragePolicySuite storagePolicySuite, + Set excludedNodes) { + } + } + + private static class ReplicationWork extends BlockRecoveryWork { + + public ReplicationWork(BlockInfo block, + BlockCollection bc, + DatanodeDescriptor[] srcNodes, + List containingNodes, + List liveReplicaStorages, + int additionalReplRequired, + int priority) { + super(block, bc, srcNodes, containingNodes, + liveReplicaStorages, additionalReplRequired, priority); + LOG.debug("Creating a ReplicationWork to recover " + block); + } + + protected void chooseTargets(BlockPlacementPolicy blockplacement, + BlockStoragePolicySuite storagePolicySuite, + Set excludedNodes) { + assert srcNodes.length > 0 + : "At least 1 source node should have been selected"; + try { + targets = blockplacement.chooseTarget(bc.getName(), + additionalReplRequired, srcNodes[0], liveReplicaStorages, false, + excludedNodes, block.getNumBytes(), + storagePolicySuite.getPolicy(bc.getStoragePolicyID())); + } finally { + srcNodes[0].decrementPendingReplicationWithoutTargets(); + } + } + } + + private static class ErasureCodingWork extends BlockRecoveryWork { + + private short[] missingBlockIndicies = null; + + public ErasureCodingWork(BlockInfo block, + BlockCollection bc, + DatanodeDescriptor[] srcNodes, + List containingNodes, + List liveReplicaStorages, + int additionalReplRequired, + int priority) { + super(block, bc, srcNodes, containingNodes, + liveReplicaStorages, additionalReplRequired, priority); + LOG.debug("Creating an ErasureCodingWork to recover " + block); + } + + public short[] getMissingBlockIndicies() { + return missingBlockIndicies; + } + + public void setMissingBlockIndices(short[] missingBlockIndicies) { + this.missingBlockIndicies = missingBlockIndicies; + } + + protected void chooseTargets(BlockPlacementPolicy blockplacement, BlockStoragePolicySuite storagePolicySuite, Set excludedNodes) { try { + // TODO: new placement policy for EC considering multiple writers targets = blockplacement.chooseTarget(bc.getName(), - additionalReplRequired, srcNode, liveReplicaStorages, false, + additionalReplRequired, srcNodes[0], liveReplicaStorages, false, excludedNodes, block.getNumBytes(), storagePolicySuite.getPolicy(bc.getStoragePolicyID())); } finally { - srcNode.decrementPendingReplicationWithoutTargets(); } } } 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 b7a3489b08957..7bc5e7e48a139 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 @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Queue; import java.util.Set; +import java.util.Arrays; import com.google.common.annotations.VisibleForTesting; @@ -41,6 +42,7 @@ 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.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; @@ -97,6 +99,33 @@ public static class BlockTargetPair { } } + /** Block and targets pair */ + @InterfaceAudience.Private + @InterfaceStability.Evolving + public static class BlockECRecoveryInfo { + public final ExtendedBlock block; + public final DatanodeDescriptor[] sources; + public final DatanodeStorageInfo[] targets; + public final short[] missingBlockIndices; + + BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources, + DatanodeStorageInfo[] targets, short[] missingBlockIndices) { + this.block = block; + this.sources = sources; + this.targets = targets; + this.missingBlockIndices = missingBlockIndices; + } + + @Override + public String toString() { + return new StringBuilder().append("BlockECRecoveryInfo(\n "). + append("Recovering ").append(block). + append(" From: ").append(Arrays.asList(sources)). + append(" To: ").append(Arrays.asList(targets)).append(")\n"). + toString(); + } + } + /** A BlockTargetPair queue. */ private static class BlockQueue { private final Queue blockq = new LinkedList(); @@ -217,12 +246,17 @@ public CachedBlocksList getPendingUncached() { private long bandwidth; /** A queue of blocks to be replicated by this datanode */ - private final BlockQueue replicateBlocks = new BlockQueue(); + private final BlockQueue replicateBlocks = + new BlockQueue<>(); + /** A queue of blocks to be erasure coded by this datanode */ + private final BlockQueue erasurecodeBlocks = + new BlockQueue<>(); /** A queue of blocks to be recovered by this datanode */ - private final BlockQueue recoverBlocks = - new BlockQueue(); + private final BlockQueue + recoverBlocks = new BlockQueue<>(); /** A set of blocks to be invalidated by this datanode */ - private final LightWeightHashSet invalidateBlocks = new LightWeightHashSet(); + private final LightWeightHashSet invalidateBlocks = + new LightWeightHashSet<>(); /* Variables for maintaining number of blocks scheduled to be written to * this storage. This count is approximate and might be slightly bigger @@ -375,6 +409,7 @@ public void clearBlockQueues() { this.invalidateBlocks.clear(); this.recoverBlocks.clear(); this.replicateBlocks.clear(); + this.erasurecodeBlocks.clear(); } // pendingCached, cached, and pendingUncached are protected by the // FSN lock. @@ -596,6 +631,20 @@ void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { replicateBlocks.offer(new BlockTargetPair(block, targets)); } + /** + * Store block erasure coding work. + */ + void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources, + DatanodeStorageInfo[] targets, short[] missingBlockIndicies) { + assert(block != null && sources != null && sources.length > 0); + BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets, + missingBlockIndicies); + erasurecodeBlocks.offer(task); + BlockManager.LOG.debug("Adding block recovery task " + task + + "to " + getName() + ", current queue size is " + + erasurecodeBlocks.size()); + } + /** * Store block recovery work. */ @@ -627,6 +676,13 @@ int getNumberOfBlocksToBeReplicated() { return PendingReplicationWithoutTargets + replicateBlocks.size(); } + /** + * The number of work items that are pending to be replicated + */ + int getNumberOfBlocksToBeErasureCoded() { + return erasurecodeBlocks.size(); + } + /** * The number of block invalidation items that are pending to * be sent to the datanode @@ -641,6 +697,10 @@ public List getReplicationCommand(int maxTransfers) { return replicateBlocks.poll(maxTransfers); } + public List getErasureCodeCommand(int maxTransfers) { + return erasurecodeBlocks.poll(maxTransfers); + } + public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { List blocks = recoverBlocks.poll(maxTransfers); if(blocks == null) @@ -841,6 +901,10 @@ public String dumpDatanode() { if (repl > 0) { sb.append(" ").append(repl).append(" blocks to be replicated;"); } + int ec = erasurecodeBlocks.size(); + if(ec > 0) { + sb.append(" ").append(ec).append(" blocks to be erasure coded;"); + } int inval = invalidateBlocks.size(); if (inval > 0) { sb.append(" ").append(inval).append(" blocks to be invalidated;"); 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 01f7972f66550..c63e657df6059 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 @@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -1349,7 +1350,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, VolumeFailureSummary volumeFailureSummary) throws IOException { synchronized (heartbeatManager) { synchronized (datanodeMap) { - DatanodeDescriptor nodeinfo = null; + DatanodeDescriptor nodeinfo; try { nodeinfo = getDatanode(nodeReg); } catch(UnregisteredNodeException e) { @@ -1387,10 +1388,10 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations(); // Skip stale nodes during recovery - not heart beated for some time (30s by default). 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]); + new ArrayList<>(storages.length); + for (DatanodeStorageInfo storage : storages) { + if (!storage.getDatanodeDescriptor().isStale(staleInterval)) { + recoveryLocations.add(storage); } } // If we are performing a truncate recovery than set recovery fields @@ -1429,7 +1430,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, return new DatanodeCommand[] { brCommand }; } - final List cmds = new ArrayList(); + final List cmds = new ArrayList<>(); //check pending replication List pendingList = nodeinfo.getReplicationCommand( maxTransfers); @@ -1437,6 +1438,13 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId, pendingList)); } + // checking pending erasure coding tasks + List pendingECList = + nodeinfo.getErasureCodeCommand(maxTransfers); + if (pendingECList != null) { + cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_CODEC, + pendingECList)); + } //check block invalidation Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit); if (blks != null) { 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 88ba2cd90ef3a..9489f8645d012 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 @@ -419,7 +419,8 @@ public short getPreferredBlockReplication() { } max = maxInSnapshot > max ? maxInSnapshot : max; } - return max; + return isStriped()? + HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max; } /** Set the replication factor of this file. */ @@ -1107,11 +1108,12 @@ boolean isBlockInLatestSnapshot(BlockInfoContiguous block) { Arrays.asList(snapshotBlocks).contains(block); } - @VisibleForTesting /** * @return true if the file is in the striping layout. */ - // TODO: move erasure coding policy to file XAttr (HDFS-7337) + @VisibleForTesting + @Override + // TODO: move erasure coding policy to file XAttr public boolean isStriped() { return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java new file mode 100644 index 0000000000000..f7f02fdea41c2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java @@ -0,0 +1,63 @@ +/** + * 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.protocol; + +import com.google.common.base.Joiner; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo; + +import java.util.Collection; + +/** + * A BlockECRecoveryCommand is an instruction to a DataNode to reconstruct a + * striped block group with missing blocks. + * + * Upon receiving this command, the DataNode pulls data from other DataNodes + * hosting blocks in this group and reconstructs the lost blocks through codec + * calculation. + * + * After the reconstruction, the DataNode pushes the reconstructed blocks to + * their final destinations if necessary (e.g., the destination is different + * from the reconstruction node, or multiple blocks in a group are to be + * reconstructed). + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BlockECRecoveryCommand extends DatanodeCommand { + final Collection ecTasks; + + /** + * Create BlockECRecoveryCommand from a collection of + * {@link BlockECRecoveryInfo}, each representing a recovery task + */ + public BlockECRecoveryCommand(int action, + Collection blockECRecoveryInfoList) { + super(action); + this.ecTasks = blockECRecoveryInfoList; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("BlockECRecoveryCommand(\n "); + Joiner.on("\n ").appendTo(sb, ecTasks); + sb.append("\n)"); + return sb.toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java index a3b6004644b9b..b8ac165db5935 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java @@ -76,6 +76,7 @@ public interface DatanodeProtocol { final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth final static int DNA_CACHE = 9; // cache blocks final static int DNA_UNCACHE = 10; // uncache blocks + final static int DNA_CODEC = 11; // uncache blocks /** * Register Datanode. 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 148135bae9727..e25ee31643e76 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 @@ -161,7 +161,7 @@ public static int computeInvalidationWork(BlockManager bm) { */ public static int computeAllPendingWork(BlockManager bm) { int work = computeInvalidationWork(bm); - work += bm.computeReplicationWork(Integer.MAX_VALUE); + work += bm.computeBlockRecoveryWork(Integer.MAX_VALUE); return work; } 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 080f693628116..a33a4e366f67d 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 @@ -453,8 +453,8 @@ private DatanodeStorageInfo[] scheduleSingleReplication(BlockInfo block) { assertEquals("Block not initially pending replication", 0, bm.pendingReplications.getNumReplicas(block)); assertEquals( - "computeReplicationWork should indicate replication is needed", 1, - bm.computeReplicationWorkForBlocks(list_all)); + "computeBlockRecoveryWork should indicate replication is needed", 1, + bm.computeRecoveryWorkForBlocks(list_all)); assertTrue("replication is pending after work is computed", bm.pendingReplications.getNumReplicas(block) > 0); @@ -508,22 +508,22 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception { assertNotNull("Chooses source node for a highest-priority replication" + " even if all available source nodes have reached their replication" + " limits below the hard limit.", - bm.chooseSourceDatanode( - aBlock, + bm.chooseSourceDatanodes( + bm.getStoredBlock(aBlock), cntNodes, liveNodes, new NumberReplicas(), - UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)); + new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]); assertNull("Does not choose a source node for a less-than-highest-priority" + " replication since all available source nodes have reached" + " their replication limits.", - bm.chooseSourceDatanode( - aBlock, + bm.chooseSourceDatanodes( + bm.getStoredBlock(aBlock), cntNodes, liveNodes, new NumberReplicas(), - UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)); + new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)[0]); // Increase the replication count to test replication count > hard limit DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] }; @@ -531,12 +531,12 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception { assertNull("Does not choose a source node for a highest-priority" + " replication when all available nodes exceed the hard limit.", - bm.chooseSourceDatanode( - aBlock, + bm.chooseSourceDatanodes( + bm.getStoredBlock(aBlock), cntNodes, liveNodes, new NumberReplicas(), - UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)); + new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java new file mode 100644 index 0000000000000..d883c9b3234fd --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java @@ -0,0 +1,107 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +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.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.client.HdfsAdmin; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Iterator; + +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME; +import static org.junit.Assert.assertTrue; + +public class TestRecoverStripedBlocks { + private final short GROUP_SIZE = + HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; + private final short NUM_OF_DATANODES = GROUP_SIZE + 1; + private Configuration conf; + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private static final int BLOCK_SIZE = 1024; + private HdfsAdmin dfsAdmin; + private FSNamesystem namesystem; + private Path ECFilePath; + + @Before + public void setupCluster() throws IOException { + conf = new HdfsConfiguration(); + conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + // Large value to make sure the pending replication request can stay in + // DatanodeDescriptor.replicateBlocks before test timeout. + conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100); + // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via + // chooseUnderReplicatedBlocks at once. + conf.setInt( + DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5); + + cluster = new MiniDFSCluster.Builder(conf). + numDataNodes(NUM_OF_DATANODES).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + dfsAdmin = new HdfsAdmin(cluster.getURI(), conf); + namesystem = cluster.getNamesystem(); + ECFilePath = new Path("/ecfile"); + DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, GROUP_SIZE, 0); + dfsAdmin.setStoragePolicy(ECFilePath, EC_STORAGE_POLICY_NAME); + } + + @Test + public void testMissingStripedBlock() throws Exception { + final BlockManager bm = cluster.getNamesystem().getBlockManager(); + ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, ECFilePath); + Iterator storageInfos = + bm.blocksMap.getStorages(b.getLocalBlock()) + .iterator(); + + DatanodeDescriptor firstDn = storageInfos.next().getDatanodeDescriptor(); + Iterator it = firstDn.getBlockIterator(); + int missingBlkCnt = 0; + while (it.hasNext()) { + BlockInfo blk = it.next(); + BlockManager.LOG.debug("Block " + blk + " will be lost"); + missingBlkCnt++; + } + BlockManager.LOG.debug("Missing in total " + missingBlkCnt + " blocks"); + + bm.getDatanodeManager().removeDatanode(firstDn); + + bm.computeDatanodeWork(); + + short cnt = 0; + for (DataNode dn : cluster.getDataNodes()) { + DatanodeDescriptor dnDescriptor = + bm.getDatanodeManager().getDatanode(dn.getDatanodeUuid()); + cnt += dnDescriptor.getNumberOfBlocksToBeErasureCoded(); + } + + assertTrue("Counting the number of outstanding EC tasks", cnt == missingBlkCnt); + } +} 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 9e2e1727d83e1..1916b3cd3fcc5 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 @@ -1162,7 +1162,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() when(mockNS.hasWriteLock()).thenReturn(true); BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; - + BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); From 90d332d6be4ded4fb666670ecb009b36d7ced7db Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 20 Mar 2015 15:07:00 +0800 Subject: [PATCH 025/212] HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 +- .../rawcoder/JRSRawErasureCoderFactory.java | 34 +++++++++++++++++ .../rawcoder/RawErasureCoderFactory.java | 38 +++++++++++++++++++ .../rawcoder/XorRawErasureCoderFactory.java | 34 +++++++++++++++++ 4 files changed, 108 insertions(+), 1 deletion(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index e27ff5c1f867d..f566f0ef3ba36 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -24,4 +24,5 @@ HADOOP-11706. Refine a little bit erasure coder API. Contributed by Kai Zheng ( Kai Zheng ) - + HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng + ( Kai Zheng ) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java new file mode 100644 index 0000000000000..d6b40aac17b9e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java @@ -0,0 +1,34 @@ +/** + * 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.io.erasurecode.rawcoder; + +/** + * A raw coder factory for raw Reed-Solomon coder in Java. + */ +public class JRSRawErasureCoderFactory implements RawErasureCoderFactory { + + @Override + public RawErasureEncoder createEncoder() { + return new JRSRawEncoder(); + } + + @Override + public RawErasureDecoder createDecoder() { + return new JRSRawDecoder(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java new file mode 100644 index 0000000000000..95a1cfe383a8d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java @@ -0,0 +1,38 @@ +/** + * 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.io.erasurecode.rawcoder; + +/** + * Raw erasure coder factory that can be used to create raw encoder and decoder. + * It helps in configuration since only one factory class is needed to be + * configured. + */ +public interface RawErasureCoderFactory { + + /** + * Create raw erasure encoder. + * @return raw erasure encoder + */ + public RawErasureEncoder createEncoder(); + + /** + * Create raw erasure decoder. + * @return raw erasure decoder + */ + public RawErasureDecoder createDecoder(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java new file mode 100644 index 0000000000000..751d16f9e7967 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java @@ -0,0 +1,34 @@ +/** + * 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.io.erasurecode.rawcoder; + +/** + * A raw coder factory for raw XOR coder. + */ +public class XorRawErasureCoderFactory implements RawErasureCoderFactory { + + @Override + public RawErasureEncoder createEncoder() { + return new XorRawEncoder(); + } + + @Override + public RawErasureDecoder createDecoder() { + return new XorRawDecoder(); + } +} From df297245a74be1a1baa1a34740dcd856eef22ea4 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 20 Mar 2015 19:15:52 +0800 Subject: [PATCH 026/212] HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/fs/CommonConfigurationKeys.java | 15 +++ .../coder/AbstractErasureCoder.java | 65 +++++++++++++ .../coder/AbstractErasureDecoder.java | 6 +- .../coder/AbstractErasureEncoder.java | 6 +- .../erasurecode/coder/RSErasureDecoder.java | 83 +++++++++++++++++ .../erasurecode/coder/RSErasureEncoder.java | 47 ++++++++++ .../erasurecode/coder/XorErasureDecoder.java | 2 +- .../erasurecode/coder/XorErasureEncoder.java | 2 +- .../erasurecode/coder/TestRSErasureCoder.java | 92 +++++++++++++++++++ 10 files changed, 315 insertions(+), 6 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index f566f0ef3ba36..b69e69adfa77b 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -26,3 +26,6 @@ HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng ( Kai Zheng ) + + HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng + ( Kai Zheng ) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 272146662ce89..bd2a24b022044 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -136,6 +136,21 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final boolean IO_COMPRESSION_CODEC_LZ4_USELZ4HC_DEFAULT = false; + /** + * Erasure Coding configuration family + */ + + /** Supported erasure codec classes */ + public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs"; + + /** Use XOR raw coder when possible for the RS codec */ + public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY = + "io.erasurecode.codec.rs.usexor"; + + /** Raw coder factory for the RS codec */ + public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY = + "io.erasurecode.codec.rs.rawcoder"; + /** * Service Authorization */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index 8d3bc3405041d..0e4de8902a90a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -17,7 +17,12 @@ */ package org.apache.hadoop.io.erasurecode.coder; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; /** * A common class of basic facilities to be shared by encoder and decoder @@ -31,6 +36,66 @@ public abstract class AbstractErasureCoder private int numParityUnits; private int chunkSize; + /** + * Create raw decoder using the factory specified by rawCoderFactoryKey + * @param rawCoderFactoryKey + * @return raw decoder + */ + protected RawErasureDecoder createRawDecoder(String rawCoderFactoryKey) { + RawErasureCoder rawCoder = createRawCoder(getConf(), + rawCoderFactoryKey, false); + return (RawErasureDecoder) rawCoder; + } + + /** + * Create raw encoder using the factory specified by rawCoderFactoryKey + * @param rawCoderFactoryKey + * @return raw encoder + */ + protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) { + RawErasureCoder rawCoder = createRawCoder(getConf(), + rawCoderFactoryKey, true); + return (RawErasureEncoder) rawCoder; + } + + /** + * Create raw coder using specified conf and raw coder factory key. + * @param conf + * @param rawCoderFactoryKey + * @param isEncoder + * @return raw coder + */ + protected static RawErasureCoder createRawCoder(Configuration conf, + String rawCoderFactoryKey, boolean isEncoder) { + + if (conf == null) { + return null; + } + + Class factClass = null; + factClass = conf.getClass(rawCoderFactoryKey, + factClass, RawErasureCoderFactory.class); + + if (factClass == null) { + return null; + } + + RawErasureCoderFactory fact; + try { + fact = factClass.newInstance(); + } catch (InstantiationException e) { + throw new RuntimeException("Failed to create raw coder", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Failed to create raw coder", e); + } + + if (fact != null) { + return isEncoder ? fact.createEncoder() : fact.createDecoder(); + } + + return null; + } + @Override public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java index df71528e20deb..54a6d1ed7a701 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java @@ -30,7 +30,8 @@ public abstract class AbstractErasureDecoder extends AbstractErasureCoder @Override public ErasureCodingStep decode(ECBlockGroup blockGroup) { - return performDecoding(blockGroup); + // We may have more than this when considering complicate cases. HADOOP-11550 + return prepareDecodingStep(blockGroup); } /** @@ -38,7 +39,8 @@ public ErasureCodingStep decode(ECBlockGroup blockGroup) { * @param blockGroup * @return decoding step for caller to do the real work */ - protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup); + protected abstract ErasureCodingStep prepareDecodingStep( + ECBlockGroup blockGroup); /** * We have all the data blocks and parity blocks as input blocks for diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java index 80b987513f9c3..09b31e5bd24df 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java @@ -30,7 +30,8 @@ public abstract class AbstractErasureEncoder extends AbstractErasureCoder @Override public ErasureCodingStep encode(ECBlockGroup blockGroup) { - return performEncoding(blockGroup); + // We may have more than this when considering complicate cases. HADOOP-11550 + return prepareEncodingStep(blockGroup); } /** @@ -38,7 +39,8 @@ public ErasureCodingStep encode(ECBlockGroup blockGroup) { * @param blockGroup * @return encoding step for caller to do the real work */ - protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup); + protected abstract ErasureCodingStep prepareEncodingStep( + ECBlockGroup blockGroup); protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { return blockGroup.getDataBlocks(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java new file mode 100644 index 0000000000000..ba32f048aded0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -0,0 +1,83 @@ +package org.apache.hadoop.io.erasurecode.coder; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder; + +/** + * Reed-Solomon erasure decoder that decodes a block group. + * + * It implements {@link ErasureDecoder}. + */ +public class RSErasureDecoder extends AbstractErasureDecoder { + private RawErasureDecoder rsRawDecoder; + private RawErasureDecoder xorRawDecoder; + private boolean useXorWhenPossible = true; + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + + if (conf != null) { + this.useXorWhenPossible = conf.getBoolean( + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true); + } + } + + @Override + protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { + + RawErasureDecoder rawDecoder; + + ECBlock[] inputBlocks = getInputBlocks(blockGroup); + ECBlock[] outputBlocks = getOutputBlocks(blockGroup); + + /** + * Optimization: according to some benchmark, when only one block is erased + * and to be recovering, the most simple XOR scheme can be much efficient. + * We will have benchmark tests to verify this opt is effect or not. + */ + if (outputBlocks.length == 1 && useXorWhenPossible) { + rawDecoder = checkCreateXorRawDecoder(); + } else { + rawDecoder = checkCreateRSRawDecoder(); + } + + return new ErasureDecodingStep(inputBlocks, + getErasedIndexes(inputBlocks), outputBlocks, rawDecoder); + } + + private RawErasureDecoder checkCreateRSRawDecoder() { + if (rsRawDecoder == null) { + rsRawDecoder = createRawDecoder( + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY); + if (rsRawDecoder == null) { + rsRawDecoder = new JRSRawDecoder(); + } + rsRawDecoder.initialize(getNumDataUnits(), + getNumParityUnits(), getChunkSize()); + } + return rsRawDecoder; + } + + private RawErasureDecoder checkCreateXorRawDecoder() { + if (xorRawDecoder == null) { + xorRawDecoder = new XorRawDecoder(); + xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize()); + } + return xorRawDecoder; + } + + @Override + public void release() { + if (xorRawDecoder != null) { + xorRawDecoder.release(); + } else if (rsRawDecoder != null) { + rsRawDecoder.release(); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java new file mode 100644 index 0000000000000..430749d3de161 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java @@ -0,0 +1,47 @@ +package org.apache.hadoop.io.erasurecode.coder; + +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; + +/** + * Reed-Solomon erasure encoder that encodes a block group. + * + * It implements {@link ErasureEncoder}. + */ +public class RSErasureEncoder extends AbstractErasureEncoder { + private RawErasureEncoder rawEncoder; + + @Override + protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { + + RawErasureEncoder rawEncoder = checkCreateRSRawEncoder(); + + ECBlock[] inputBlocks = getInputBlocks(blockGroup); + + return new ErasureEncodingStep(inputBlocks, + getOutputBlocks(blockGroup), rawEncoder); + } + + private RawErasureEncoder checkCreateRSRawEncoder() { + if (rawEncoder == null) { + rawEncoder = createRawEncoder( + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY); + if (rawEncoder == null) { + rawEncoder = new JRSRawEncoder(); + } + rawEncoder.initialize(getNumDataUnits(), + getNumParityUnits(), getChunkSize()); + } + return rawEncoder; + } + + @Override + public void release() { + if (rawEncoder != null) { + rawEncoder.release(); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java index 9a6f6e2df9941..33f53866ff0f8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java @@ -30,7 +30,7 @@ public class XorErasureDecoder extends AbstractErasureDecoder { @Override - protected ErasureCodingStep performDecoding(final ECBlockGroup blockGroup) { + protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { // May be configured RawErasureDecoder rawDecoder = new XorRawDecoder(); rawDecoder.initialize(getNumDataUnits(), diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java index db6e3db16e6f2..f8d67c342c92e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java @@ -30,7 +30,7 @@ public class XorErasureEncoder extends AbstractErasureEncoder { @Override - protected ErasureCodingStep performEncoding(final ECBlockGroup blockGroup) { + protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { // May be configured RawErasureEncoder rawEncoder = new XorRawEncoder(); rawEncoder.initialize(getNumDataUnits(), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java new file mode 100644 index 0000000000000..8a7561c1e2faa --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java @@ -0,0 +1,92 @@ +/** + * 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.io.erasurecode.coder; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawErasureCoderFactory; +import org.junit.Before; +import org.junit.Test; + +/** + * Test Reed-Solomon encoding and decoding. + */ +public class TestRSErasureCoder extends TestErasureCoderBase { + + @Before + public void setup() { + this.encoderClass = RSErasureEncoder.class; + this.decoderClass = RSErasureDecoder.class; + + this.numDataUnits = 10; + this.numParityUnits = 1; + + this.numChunksInBlock = 10; + } + + @Test + public void testCodingNoDirectBuffer_10x4() { + prepare(null, 10, 4, null); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_10x4() { + prepare(null, 10, 4, null); + testCoding(true); + } + + @Test + public void testCodingDirectBufferWithConf_10x4() { + /** + * This tests if the two configuration items work or not. + */ + Configuration conf = new Configuration(); + conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + JRSRawErasureCoderFactory.class.getCanonicalName()); + conf.setBoolean( + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false); + prepare(conf, 10, 4, null); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasure_of_2_4() { + prepare(null, 10, 4, new int[] {2, 4}); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_all() { + prepare(null, 10, 4, new int[] {0, 1, 2, 3}); + testCoding(true); + } + + @Test + public void testCodingNoDirectBuffer_3x3() { + prepare(null, 3, 3, null); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_3x3() { + prepare(null, 3, 3, null); + testCoding(true); + } + +} From 544f75d6512fefd0e36f24a35e6b7472ca7bf301 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 23 Mar 2015 11:25:40 -0700 Subject: [PATCH 027/212] HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435 and HDFS-7930 (this commit is for HDFS-7930 only) --- .../hadoop/hdfs/server/blockmanagement/BlockManager.java | 7 ++++--- .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 7 ++++--- .../org/apache/hadoop/hdfs/server/namenode/INodeFile.java | 4 ++-- 3 files changed, 10 insertions(+), 8 deletions(-) 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 bbead63f31317..09c4284b5d4d8 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 @@ -2135,17 +2135,18 @@ private Collection processReport( * Mark block replicas as corrupt except those on the storages in * newStorages list. */ - public void markBlockReplicasAsCorrupt(BlockInfoContiguous block, + public void markBlockReplicasAsCorrupt(Block oldBlock, + BlockInfo block, long oldGenerationStamp, long oldNumBytes, DatanodeStorageInfo[] newStorages) throws IOException { assert namesystem.hasWriteLock(); BlockToMarkCorrupt b = null; if (block.getGenerationStamp() != oldGenerationStamp) { - b = new BlockToMarkCorrupt(block, oldGenerationStamp, + b = new BlockToMarkCorrupt(oldBlock, block, oldGenerationStamp, "genstamp does not match " + oldGenerationStamp + " : " + block.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); } else if (block.getNumBytes() != oldNumBytes) { - b = new BlockToMarkCorrupt(block, + b = new BlockToMarkCorrupt(oldBlock, block, "length does not match " + oldNumBytes + " : " + block.getNumBytes(), Reason.SIZE_MISMATCH); } else { 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 29215251221cd..ecb6ee1b4f4d6 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 @@ -2630,7 +2630,7 @@ private QuotaCounts verifyQuotaForUCBlock(INodeFile file, INodesInPath iip) /** 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(); @@ -3854,8 +3854,9 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, } else { iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos); if (closeFile) { - blockManager.markBlockReplicasAsCorrupt(storedBlock, - oldGenerationStamp, oldNumBytes, trimmedStorageInfos); + blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(), + storedBlock, oldGenerationStamp, oldNumBytes, + trimmedStorageInfos); } } } 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 9489f8645d012..d5a809c8fdec7 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 @@ -982,7 +982,7 @@ public long collectBlocksBeyondMax(final long max, void computeQuotaDeltaForTruncate( long newLength, BlockStoragePolicy bsps, QuotaCounts delta) { - final BlockInfoContiguous[] blocks = getBlocks(); + final BlockInfo[] blocks = getBlocks(); if (blocks == null || blocks.length == 0) { return; } @@ -1001,7 +1001,7 @@ void computeQuotaDeltaForTruncate( 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 From 46dac3595fa2e2c14290154c3c12ea799ee5043d Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 23 Mar 2015 15:06:53 -0700 Subject: [PATCH 028/212] HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks. Contributed by GAO Rui. --- .../server/blockmanagement/BlockIdManager.java | 6 ++++++ .../server/blockmanagement/BlockManager.java | 12 +++++++----- .../hdfs/server/blockmanagement/BlocksMap.java | 2 +- .../hdfs/server/namenode/FSNamesystem.java | 17 ++++++++++++----- .../hadoop/hdfs/server/namenode/SafeMode.java | 5 +++-- .../org/apache/hadoop/hdfs/TestSafeMode.java | 15 +++++++++++++-- 6 files changed, 42 insertions(+), 15 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index 8a71f18715a8c..fc82d5da4c225 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -234,6 +234,12 @@ public static boolean isStripedBlockID(long id) { return id < 0; } + /** + * The last 4 bits of HdfsConstants.BLOCK_GROUP_INDEX_MASK(15) is 1111, + * so the last 4 bits of (~HdfsConstants.BLOCK_GROUP_INDEX_MASK) is 0000 + * and the other 60 bits are 1. Group ID is the first 60 bits of any + * data/parity block id in the same striped block group. + */ public static long convertToStripedID(long id) { return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK); } 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 09c4284b5d4d8..079d218877fe1 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 @@ -687,8 +687,10 @@ private BlockInfo completeBlock(final BlockCollection bc, // a "forced" completion when a file is getting closed by an // OP_CLOSE edit on the standby). namesystem.adjustSafeModeBlockTotals(0, 1); + final int minStorage = curBlock.isStriped() ? + ((BlockInfoStriped) curBlock).getDataBlockNum() : minReplication; namesystem.incrementSafeBlockCount( - Math.min(numNodes, minReplication)); + Math.min(numNodes, minStorage), curBlock); // replace block in the blocksMap return blocksMap.replaceBlock(completeBlock); @@ -2234,7 +2236,7 @@ private void processFirstBlockReport( // refer HDFS-5283 if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) { int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock); - namesystem.incrementSafeBlockCount(numOfReplicas); + namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock); } //and fall through to next clause } @@ -2622,14 +2624,14 @@ && checkMinStorage(storedBlock, numCurrentReplica)) { // only complete blocks are counted towards that. // In the case that the block just became complete above, completeBlock() // handles the safe block count maintenance. - namesystem.incrementSafeBlockCount(numCurrentReplica); + namesystem.incrementSafeBlockCount(numCurrentReplica, storedBlock); } } /** * Modify (block-->datanode) map. Remove block from set of * needed replications if this takes care of the problem. - * @return the block that is stored in blockMap. + * @return the block that is stored in blocksMap. */ private Block addStoredBlock(final BlockInfo block, final Block reportedBlock, @@ -2698,7 +2700,7 @@ private Block addStoredBlock(final BlockInfo block, // Is no-op if not in safe mode. // In the case that the block just became complete above, completeBlock() // handles the safe block count maintenance. - namesystem.incrementSafeBlockCount(numCurrentReplica); + namesystem.incrementSafeBlockCount(numCurrentReplica, storedBlock); } // if file is under construction, then done for now 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 b093e5b143db4..9caf47c73fe79 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 @@ -140,7 +140,7 @@ void removeBlock(Block block) { } } - /** Returns the block object it it exists in the map. */ + /** Returns the block object if it exists in the map. */ BlockInfo getStoredBlock(Block b) { return blocks.get(b); } 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 ecb6ee1b4f4d6..0460f5ce0a2f0 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 @@ -214,6 +214,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; @@ -4822,10 +4823,16 @@ private synchronized void setBlockTotal(int total) { /** * Increment number of safe blocks if current block has * reached minimal replication. - * @param replication current replication + * @param storageNum current number of replicas or number of internal blocks + * of a striped block group + * @param storedBlock current storedBlock which is either a + * BlockInfoContiguous or a BlockInfoStriped */ - private synchronized void incrementSafeBlockCount(short replication) { - if (replication == safeReplication) { + private synchronized void incrementSafeBlockCount(short storageNum, + BlockInfo storedBlock) { + final int safe = storedBlock.isStriped() ? + ((BlockInfoStriped) storedBlock).getDataBlockNum() : safeReplication; + if (storageNum == safe) { this.blockSafe++; // Report startup progress only if we haven't completed startup yet. @@ -5118,12 +5125,12 @@ private boolean shouldPopulateReplQueues() { } @Override - public void incrementSafeBlockCount(int replication) { + public void incrementSafeBlockCount(int storageNum, BlockInfo storedBlock) { // safeMode is volatile, and may be set to null at any time SafeModeInfo safeMode = this.safeMode; if (safeMode == null) return; - safeMode.incrementSafeBlockCount((short)replication); + safeMode.incrementSafeBlockCount((short) storageNum, storedBlock); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java index 0debb1f77223a..e26e7270b06b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java @@ -45,9 +45,10 @@ public interface SafeMode { /** * Increment number of blocks that reached minimal replication. - * @param replication current replication + * @param replication current replication + * @param storedBlock current stored Block */ - public void incrementSafeBlockCount(int replication); + public void incrementSafeBlockCount(int replication, BlockInfo storedBlock); /** Decrement number of blocks that reached minimal replication. */ public void decrementSafeBlockCount(BlockInfo b); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java index 80fe9eead5491..a43e37173f566 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java @@ -552,7 +552,18 @@ public void testSafeModeWhenZeroBlockLocations() throws IOException { if(cluster!= null) cluster.shutdown(); } } - + + //TODO : test should be added to check safeMode with stripedBloks after stripedBlock related functions have been added in class MiniDFSCluster + @Test + public void testSafeModeWithCorruptSripedBlock() throws IOException { + try { + + } finally { + if(fs != null) fs.close(); + if(cluster!= null) cluster.shutdown(); + } + } + void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOException { FileStatus stat = fs.getFileStatus(fileName); try { @@ -560,7 +571,7 @@ void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOExceptio } catch (SafeModeException e) { assertTrue("Should have not got safemode exception", false); } catch (RemoteException re) { - assertTrue("Should have not got safemode exception", false); + assertTrue("Should have not got remote exception", false); } } } From d0d75a833907f6cf723a42a007ca04e0004a8e52 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 23 Mar 2015 15:10:10 -0700 Subject: [PATCH 029/212] HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage. Contributed by Hui Zheng. --- .../blockmanagement/BlockInfoStriped.java | 11 +- .../hdfs/server/namenode/FSImageFormat.java | 62 ++++++-- .../server/namenode/FSImageSerialization.java | 78 ++++++--- .../blockmanagement/TestBlockInfoStriped.java | 34 ++++ .../hdfs/server/namenode/TestFSImage.java | 148 +++++++++++++++++- 5 files changed, 300 insertions(+), 33 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index cef83185b168d..30b5ee715f122 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -20,6 +20,8 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; +import java.io.DataOutput; +import java.io.IOException; /** * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. @@ -206,6 +208,13 @@ public int numNodes() { return num; } + @Override + public void write(DataOutput out) throws IOException { + out.writeShort(dataBlockNum); + out.writeShort(parityBlockNum); + super.write(out); + } + /** * Convert a complete block to an under construction block. * @return BlockInfoUnderConstruction - an under construction block. @@ -215,7 +224,7 @@ public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( final BlockInfoStripedUnderConstruction ucBlock; if(isComplete()) { ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(), - getParityBlockNum(), s, targets); + getParityBlockNum(), s, targets); ucBlock.setBlockCollection(getBlockCollection()); } else { // the block is already under construction 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 aef0b2811b924..0582a0a361636 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 @@ -48,13 +48,16 @@ import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.Block; 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; @@ -755,16 +758,31 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, atime = in.readLong(); } final long blockSize = in.readLong(); + final boolean isStriped = NameNodeLayoutVersion.supports( + NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion) + && (in.readBoolean()); final int numBlocks = in.readInt(); if (numBlocks >= 0) { // file // read blocks - BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks]; - for (int j = 0; j < numBlocks; j++) { - blocks[j] = new BlockInfoContiguous(replication); - blocks[j].readFields(in); + Block[] blocks; + if (isStriped) { + blocks = new Block[numBlocks]; + for (int j = 0; j < numBlocks; j++) { + short dataBlockNum = in.readShort(); + short parityBlockNum = in.readShort(); + blocks[j] = new BlockInfoStriped(new Block(), + dataBlockNum, parityBlockNum); + blocks[j].readFields(in); + } + } else { + blocks = new BlockInfoContiguous[numBlocks]; + for (int j = 0; j < numBlocks; j++) { + blocks[j] = new BlockInfoContiguous(replication); + blocks[j].readFields(in); + } } String clientName = ""; @@ -783,9 +801,18 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, clientMachine = FSImageSerialization.readString(in); // convert the last block to BlockUC if (blocks.length > 0) { - BlockInfoContiguous lastBlk = blocks[blocks.length - 1]; - blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction( - lastBlk, replication); + Block lastBlk = blocks[blocks.length - 1]; + if (isStriped){ + BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk; + blocks[blocks.length - 1] + = new BlockInfoStripedUnderConstruction(lastBlk, + lastStripedBlk.getDataBlockNum(), + lastStripedBlk.getParityBlockNum()); + } else { + blocks[blocks.length - 1] + = new BlockInfoContiguousUnderConstruction(lastBlk, + replication); + } } } } @@ -798,14 +825,25 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, counter.increment(); } - final INodeFile file = new INodeFile(inodeId, localName, permissions, - modificationTime, atime, blocks, replication, blockSize); + INodeFile file; + if (isStriped) { + file = new INodeFile(inodeId, localName, permissions, modificationTime, + atime, new BlockInfoContiguous[0], (short) 0, blockSize); + file.addStripedBlocksFeature(); + for (Block block : blocks) { + file.getStripedBlocksFeature().addBlock((BlockInfoStriped) block); + } + } else { + file = new INodeFile(inodeId, localName, permissions, + modificationTime, atime, (BlockInfoContiguous[]) blocks, + replication, blockSize); + } if (underConstruction) { file.toUnderConstruction(clientName, clientMachine); } - return fileDiffs == null ? file : new INodeFile(file, fileDiffs); - } else if (numBlocks == -1) { - //directory + return fileDiffs == null ? file : new INodeFile(file, fileDiffs); + } else if (numBlocks == -1) { + //directory //read quotas final long nsQuota = in.readLong(); 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 1888d878cb8f8..1e58858cdfac8 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 @@ -35,6 +35,8 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap; @@ -124,21 +126,48 @@ static INodeFile readINodeUnderConstruction( short blockReplication = in.readShort(); long modificationTime = in.readLong(); long preferredBlockSize = in.readLong(); + final boolean isStriped = NameNodeLayoutVersion.supports( + NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion) + && (in.readBoolean()); int numBlocks = in.readInt(); - BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks]; - Block blk = new Block(); - int i = 0; - for (; i < numBlocks-1; i++) { - blk.readFields(in); - blocks[i] = new BlockInfoContiguous(blk, blockReplication); - } - // last block is UNDER_CONSTRUCTION - if(numBlocks > 0) { - blk.readFields(in); - blocks[i] = new BlockInfoContiguousUnderConstruction( - blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); + + final BlockInfoContiguous[] blocksContiguous; + BlockInfoStriped[] blocksStriped = null; + if (isStriped) { + blocksContiguous = new BlockInfoContiguous[0]; + blocksStriped = new BlockInfoStriped[numBlocks]; + int i = 0; + for (; i < numBlocks - 1; i++) { + short dataBlockNum = in.readShort(); + short parityBlockNum = in.readShort(); + blocksStriped[i] = new BlockInfoStriped(new Block(), dataBlockNum, + parityBlockNum); + blocksStriped[i].readFields(in); + } + if (numBlocks > 0) { + short dataBlockNum = in.readShort(); + short parityBlockNum = in.readShort(); + blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(), + dataBlockNum, parityBlockNum, BlockUCState.UNDER_CONSTRUCTION, null); + blocksStriped[i].readFields(in); + } + } else { + blocksContiguous = new BlockInfoContiguous[numBlocks]; + Block blk = new Block(); + int i = 0; + for (; i < numBlocks-1; i++) { + blk.readFields(in); + blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication); + } + // last block is UNDER_CONSTRUCTION + if(numBlocks > 0) { + blk.readFields(in); + blocksContiguous[i] = new BlockInfoContiguousUnderConstruction( + blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); + } } + PermissionStatus perm = PermissionStatus.read(in); String clientName = readString(in); String clientMachine = readString(in); @@ -150,8 +179,19 @@ static INodeFile readINodeUnderConstruction( // Images in the pre-protobuf format will not have the lazyPersist flag, // so it is safe to pass false always. - INodeFile file = new INodeFile(inodeId, name, perm, modificationTime, - modificationTime, blocks, blockReplication, preferredBlockSize); + INodeFile file; + if (isStriped) { + file = new INodeFile(inodeId, name, perm, modificationTime, + modificationTime, blocksContiguous, (short) 0, preferredBlockSize); + file.addStripedBlocksFeature(); + for (int i = 0; i < numBlocks; i++) { + file.getStripedBlocksFeature().addBlock(blocksStriped[i]); + } + } else { + file = new INodeFile(inodeId, name, perm, modificationTime, + modificationTime, blocksContiguous, blockReplication, + preferredBlockSize); + } file.toUnderConstruction(clientName, clientMachine); return file; } @@ -166,7 +206,8 @@ static void writeINodeUnderConstruction(DataOutputStream out, INodeFile cons, out.writeShort(cons.getFileReplication()); out.writeLong(cons.getModificationTime()); out.writeLong(cons.getPreferredBlockSize()); - + // whether the file has striped blocks + out.writeBoolean(cons.isWithStripedBlocks()); writeBlocks(cons.getBlocks(), out); cons.getPermissionStatus().write(out); @@ -179,9 +220,9 @@ static void writeINodeUnderConstruction(DataOutputStream out, INodeFile cons, /** * Serialize a {@link INodeFile} node - * @param node The node to write + * @param file The node to write * @param out The {@link DataOutputStream} where the fields are written - * @param writeBlock Whether to write block information + * @param writeUnderConstruction Whether to write block information */ public static void writeINodeFile(INodeFile file, DataOutput out, boolean writeUnderConstruction) throws IOException { @@ -191,7 +232,8 @@ public static void writeINodeFile(INodeFile file, DataOutput out, out.writeLong(file.getModificationTime()); out.writeLong(file.getAccessTime()); out.writeLong(file.getPreferredBlockSize()); - + // whether the file has striped blocks + out.writeBoolean(file.isWithStripedBlocks()); writeBlocks(file.getBlocks(), out); SnapshotFSImageFormat.saveFileDiffList(file, out); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java index 74ddac08a2b7c..c4db5d4d37447 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -25,8 +25,16 @@ import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.ByteArrayOutputStream; +import java.nio.ByteBuffer; + import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** * Test {@link BlockInfoStriped} @@ -216,4 +224,30 @@ public void testReplaceBlock() { Assert.assertNull(newBlockInfo.getNext()); } } + + @Test + public void testWrite() { + long blkID = 1; + long numBytes = 1; + long generationStamp = 1; + short dataBlockNum = 6; + short parityBlockNum = 3; + ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE/Byte.SIZE*3 + + Short.SIZE/Byte.SIZE*2); + byteBuffer.putShort(dataBlockNum).putShort(parityBlockNum) + .putLong(blkID).putLong(numBytes).putLong(generationStamp); + + ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); + DataOutput out = new DataOutputStream(byteStream); + BlockInfoStriped blk = new BlockInfoStriped(new Block(1,1,1), + (short)6,(short)3); + try { + blk.write(out); + } catch(Exception ex) { + fail("testWrite error:" + ex.getMessage()); + } + assertEquals(byteBuffer.array().length, byteStream.toByteArray().length); + assertArrayEquals(byteBuffer.array(), byteStream.toByteArray()); + } + } 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 25c0bcfc04fd8..21df6a993bdc7 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 @@ -17,18 +17,28 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.File; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.EnumSet; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.junit.Assert; +import org.apache.hadoop.fs.permission.PermissionStatus; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -42,8 +52,8 @@ 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.protocol.Block; 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; import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.test.GenericTestUtils; @@ -120,6 +130,140 @@ private void testPersistHelper(Configuration conf) throws IOException { } } + private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf, + boolean isUC) throws IOException{ + // contruct a INode with StripedBlock for saving and loading + long id = 123456789; + byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes(); + PermissionStatus permissionStatus = new PermissionStatus("testuser_a", + "testuser_groups", new FsPermission((short)0x755)); + long mtime = 1426222916-3600; + long atime = 1426222916; + BlockInfoContiguous[] blks = new BlockInfoContiguous[0]; + short replication = 3; + long preferredBlockSize = 128*1024*1024; + byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID; + INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime, + blks, replication, preferredBlockSize, storagePolicyID); + ByteArrayOutputStream bs = new ByteArrayOutputStream(); + file.addStripedBlocksFeature(); + + //construct StripedBlocks for the INode + BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3]; + long stripedBlkId = 10000001; + long timestamp = mtime+3600; + for (int i = 0; i < stripedBlks.length; i++) { + stripedBlks[i] = new BlockInfoStriped( + new Block(stripedBlkId + i, preferredBlockSize, timestamp), + (short) 6, (short) 3); + file.getStripedBlocksFeature().addBlock(stripedBlks[i]); + } + + final String client = "testClient"; + final String clientMachine = "testClientMachine"; + final String path = "testUnderConstructionPath"; + + //save the INode to byte array + DataOutput out = new DataOutputStream(bs); + if (isUC) { + file.toUnderConstruction(client, clientMachine); + FSImageSerialization.writeINodeUnderConstruction((DataOutputStream) out, + file, path); + } else { + FSImageSerialization.writeINodeFile(file, out, false); + } + DataInput in = new DataInputStream( + new ByteArrayInputStream(bs.toByteArray())); + + // load the INode from the byte array + INodeFile fileByLoaded; + if (isUC) { + fileByLoaded = FSImageSerialization.readINodeUnderConstruction(in, + fsn, fsn.getFSImage().getLayoutVersion()); + } else { + fileByLoaded = (INodeFile) new FSImageFormat.Loader(conf, fsn) + .loadINodeWithLocalName(false, in, false); + } + + assertEquals(id, fileByLoaded.getId() ); + assertArrayEquals(isUC ? path.getBytes() : name, + fileByLoaded.getLocalName().getBytes()); + assertEquals(permissionStatus.getUserName(), + fileByLoaded.getPermissionStatus().getUserName()); + assertEquals(permissionStatus.getGroupName(), + fileByLoaded.getPermissionStatus().getGroupName()); + assertEquals(permissionStatus.getPermission(), + fileByLoaded.getPermissionStatus().getPermission()); + assertEquals(mtime, fileByLoaded.getModificationTime()); + assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime()); + assertEquals(0, fileByLoaded.getContiguousBlocks().length); + assertEquals(0, fileByLoaded.getBlockReplication()); + assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize()); + + //check the BlockInfoStriped + BlockInfoStriped[] stripedBlksByLoaded = + fileByLoaded.getStripedBlocksFeature().getBlocks(); + assertEquals(3, stripedBlksByLoaded.length); + for (int i = 0; i < 3; i++) { + assertEquals(stripedBlks[i].getBlockId(), + stripedBlksByLoaded[i].getBlockId()); + assertEquals(stripedBlks[i].getNumBytes(), + stripedBlksByLoaded[i].getNumBytes()); + assertEquals(stripedBlks[i].getGenerationStamp(), + stripedBlksByLoaded[i].getGenerationStamp()); + assertEquals(stripedBlks[i].getDataBlockNum(), + stripedBlksByLoaded[i].getDataBlockNum()); + assertEquals(stripedBlks[i].getParityBlockNum(), + stripedBlksByLoaded[i].getParityBlockNum()); + } + + if (isUC) { + assertEquals(client, + fileByLoaded.getFileUnderConstructionFeature().getClientName()); + assertEquals(clientMachine, + fileByLoaded.getFileUnderConstructionFeature().getClientMachine()); + } + } + + /** + * Test if a INodeFile with BlockInfoStriped can be saved by + * FSImageSerialization and loaded by FSImageFormat#Loader. + */ + @Test + public void testSaveAndLoadInodeFile() throws IOException{ + Configuration conf = new Configuration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).build(); + cluster.waitActive(); + testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + /** + * Test if a INodeFileUnderConstruction with BlockInfoStriped can be + * saved and loaded by FSImageSerialization + */ + @Test + public void testSaveAndLoadInodeFileUC() throws IOException{ + // construct a INode with StripedBlock for saving and loading + Configuration conf = new Configuration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).build(); + cluster.waitActive(); + testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + /** * Ensure that the digest written by the saver equals to the digest of the * file. From ea2e60fbcc79c65ec571224bd3f57c262a5d9114 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 24 Mar 2015 11:39:36 -0700 Subject: [PATCH 030/212] HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435, HDFS-7930, HDFS-7960 (this commit is for HDFS-7960) --- .../hadoop/hdfs/server/blockmanagement/BlockManager.java | 4 ++-- .../blockmanagement/TestNameNodePrunesMissingStorages.java | 5 ++++- .../hadoop/hdfs/server/namenode/TestAddStripedBlocks.java | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) 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 079d218877fe1..363e687a3689e 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 @@ -1996,10 +1996,10 @@ private void removeZombieReplicas(BlockReportContext context, "longer exists on the DataNode.", Long.toHexString(context.getReportId()), zombie.getStorageID()); assert(namesystem.hasWriteLock()); - Iterator iter = zombie.getBlockIterator(); + Iterator iter = zombie.getBlockIterator(); int prevBlocks = zombie.numBlocks(); while (iter.hasNext()) { - BlockInfoContiguous block = iter.next(); + BlockInfo block = iter.next(); // We assume that a block can be on only one storage in a DataNode. // That's why we pass in the DatanodeDescriptor rather than the // DatanodeStorageInfo. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java index d73f63e7386a5..58c1717c31a06 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java @@ -185,9 +185,12 @@ public void testRemovingStorageDoesNotProduceZombies() throws Exception { String datanodeUuid; // Find the first storage which this block is in. try { + BlockInfo storedBlock = + cluster.getNamesystem().getBlockManager(). + getStoredBlock(block.getLocalBlock()); Iterator storageInfoIter = cluster.getNamesystem().getBlockManager(). - getStorages(block.getLocalBlock()).iterator(); + blocksMap.getStorages(storedBlock).iterator(); assertTrue(storageInfoIter.hasNext()); DatanodeStorageInfo info = storageInfoIter.next(); storageIdToRemove = info.getStorageID(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 05aec4be95b02..7d7c81e0fe405 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -269,7 +269,7 @@ public void testAddUCReplica() throws Exception { StorageBlockReport[] reports = {new StorageBlockReport(storage, bll)}; cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId), - bpId, reports); + bpId, reports, null); } BlockInfoStripedUnderConstruction ucBlock = From 9d1175b8fb18065c8a9bde486b59144ffc68ed83 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 25 Mar 2015 19:01:03 +0900 Subject: [PATCH 031/212] HDFS-7716. Add a test for BlockGroup support in FSImage. Contributed by Takuya Fukudome --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 6 ++- .../hdfs/server/namenode/TestFSImage.java | 53 +++++++++++++++++++ 2 files changed, 58 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2ef8527334aae..21e4c033da679 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -1,4 +1,8 @@ BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS HDFS-7347. Configurable erasure coding policy for individual files and - directories ( Zhe Zhang via vinayakumarb ) \ No newline at end of file + directories ( Zhe Zhang via vinayakumarb ) + + HDFS-7716. Add a test for BlockGroup support in FSImage. + (Takuya Fukudome via szetszwo) + 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 21df6a993bdc7..16f247add78b2 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 @@ -19,6 +19,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.File; @@ -31,7 +32,12 @@ import java.io.IOException; import java.util.EnumSet; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.junit.Assert; @@ -46,6 +52,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSOutputStream; +import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -378,4 +385,50 @@ public void testZeroBlockSize() throws Exception { FileUtil.fullyDelete(dfsDir); } } + + /** + * Ensure that FSImage supports BlockGroup. + */ + @Test + public void testSupportBlockGroup() throws IOException { + final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; + final int BLOCK_SIZE = 8 * 1024 * 1024; + Configuration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME); + Path file = new Path("/striped"); + FSDataOutputStream out = fs.create(file); + byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE); + out.write(bytes); + out.close(); + + fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER); + fs.saveNamespace(); + fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE); + + cluster.restartNameNodes(); + fs = cluster.getFileSystem(); + assertTrue(fs.exists(file)); + + // check the information of striped blocks + FSNamesystem fsn = cluster.getNamesystem(); + INodeFile inode = fsn.dir.getINode(file.toString()).asFile(); + FileWithStripedBlocksFeature sb = inode.getStripedBlocksFeature(); + assertNotNull(sb); + BlockInfoStriped[] blks = sb.getBlocks(); + assertEquals(1, blks.length); + assertTrue(blks[0].isStriped()); + assertEquals(HdfsConstants.NUM_DATA_BLOCKS, blks[0].getDataBlockNum()); + assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, blks[0].getParityBlockNum()); + } finally { + cluster.shutdown(); + } + } } From d9af36b9bdd1574873f38286a54611d9c8e1a1c7 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 27 Mar 2015 14:52:50 -0700 Subject: [PATCH 032/212] HADOOP-11664. Loading predefined EC schemas from configuration. Contributed by Kai Zheng. --- .../src/main/conf/ecschema-def.xml | 40 +++++ .../hadoop/fs/CommonConfigurationKeys.java | 5 + .../hadoop/io/erasurecode/SchemaLoader.java | 147 ++++++++++++++++++ .../io/erasurecode/TestSchemaLoader.java | 80 ++++++++++ 4 files changed, 272 insertions(+) create mode 100644 hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml new file mode 100644 index 0000000000000..e6194857ef2e3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml @@ -0,0 +1,40 @@ + + + + + + + + + 6 + 3 + RS + + + 10 + 4 + RS + + \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index bd2a24b022044..8a5211a70bbb9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -143,6 +143,11 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** Supported erasure codec classes */ public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs"; + public static final String IO_ERASURECODE_SCHEMA_FILE_KEY = + "io.erasurecode.schema.file"; + public static final String IO_ERASURECODE_SCHEMA_FILE_DEFAULT = + "ecschema-def.xml"; + /** Use XOR raw coder when possible for the RS codec */ public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY = "io.erasurecode.codec.rs.usexor"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java new file mode 100644 index 0000000000000..c51ed37608326 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java @@ -0,0 +1,147 @@ +/** + * 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.io.erasurecode; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.w3c.dom.*; +import org.xml.sax.SAXException; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.*; + +/** + * A EC schema loading utility that loads predefined EC schemas from XML file + */ +public class SchemaLoader { + private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName()); + + /** + * Load predefined ec schemas from configuration file. This file is + * expected to be in the XML format. + */ + public List loadSchema(Configuration conf) { + File confFile = getSchemaFile(conf); + if (confFile == null) { + LOG.warn("Not found any predefined EC schema file"); + return Collections.emptyList(); + } + + try { + return loadSchema(confFile); + } catch (ParserConfigurationException e) { + throw new RuntimeException("Failed to load schema file: " + confFile); + } catch (IOException e) { + throw new RuntimeException("Failed to load schema file: " + confFile); + } catch (SAXException e) { + throw new RuntimeException("Failed to load schema file: " + confFile); + } + } + + private List loadSchema(File schemaFile) + throws ParserConfigurationException, IOException, SAXException { + + LOG.info("Loading predefined EC schema file " + schemaFile); + + // Read and parse the schema file. + DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance(); + dbf.setIgnoringComments(true); + DocumentBuilder builder = dbf.newDocumentBuilder(); + Document doc = builder.parse(schemaFile); + Element root = doc.getDocumentElement(); + + if (!"schemas".equals(root.getTagName())) { + throw new RuntimeException("Bad EC schema config file: " + + "top-level element not "); + } + + NodeList elements = root.getChildNodes(); + List schemas = new ArrayList(); + for (int i = 0; i < elements.getLength(); i++) { + Node node = elements.item(i); + if (node instanceof Element) { + Element element = (Element) node; + if ("schema".equals(element.getTagName())) { + ECSchema schema = loadSchema(element); + schemas.add(schema); + } else { + LOG.warn("Bad element in EC schema configuration file: " + + element.getTagName()); + } + } + } + + return schemas; + } + + /** + * Path to the XML file containing predefined ec schemas. If the path is + * relative, it is searched for in the classpath. + */ + private File getSchemaFile(Configuration conf) { + String schemaFilePath = conf.get( + CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY, + CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_DEFAULT); + File schemaFile = new File(schemaFilePath); + if (! schemaFile.isAbsolute()) { + URL url = Thread.currentThread().getContextClassLoader() + .getResource(schemaFilePath); + if (url == null) { + LOG.warn(schemaFilePath + " not found on the classpath."); + schemaFile = null; + } else if (! url.getProtocol().equalsIgnoreCase("file")) { + throw new RuntimeException( + "EC predefined schema file " + url + + " found on the classpath is not on the local filesystem."); + } else { + schemaFile = new File(url.getPath()); + } + } + + return schemaFile; + } + + /** + * Loads a schema from a schema element in the configuration file + */ + private ECSchema loadSchema(Element element) { + String schemaName = element.getAttribute("name"); + Map ecOptions = new HashMap(); + NodeList fields = element.getChildNodes(); + + for (int i = 0; i < fields.getLength(); i++) { + Node fieldNode = fields.item(i); + if (fieldNode instanceof Element) { + Element field = (Element) fieldNode; + String tagName = field.getTagName(); + String value = ((Text) field.getFirstChild()).getData().trim(); + ecOptions.put(tagName, value); + } + } + + ECSchema schema = new ECSchema(schemaName, ecOptions); + return schema; + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java new file mode 100644 index 0000000000000..7bb0a9a121857 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java @@ -0,0 +1,80 @@ +/** + * 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.io.erasurecode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.junit.Test; + +import java.io.File; +import java.io.FileWriter; +import java.io.PrintWriter; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TestSchemaLoader { + + final static String TEST_DIR = new File(System.getProperty( + "test.build.data", "/tmp")).getAbsolutePath(); + + final static String SCHEMA_FILE = new File(TEST_DIR, "test-ecschema") + .getAbsolutePath(); + + @Test + public void testLoadSchema() throws Exception { + PrintWriter out = new PrintWriter(new FileWriter(SCHEMA_FILE)); + out.println(""); + out.println(""); + out.println(" "); + out.println(" 6"); + out.println(" 3"); + out.println(" RS"); + out.println(" "); + out.println(" "); + out.println(" 10"); + out.println(" 4"); + out.println(" RS"); + out.println(" "); + out.println(""); + out.close(); + + Configuration conf = new Configuration(); + conf.set(CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY, + SCHEMA_FILE); + + SchemaLoader schemaLoader = new SchemaLoader(); + List schemas = schemaLoader.loadSchema(conf); + + assertEquals(2, schemas.size()); + + ECSchema schema1 = schemas.get(0); + assertEquals("RSk6m3", schema1.getSchemaName()); + assertEquals(3, schema1.getOptions().size()); + assertEquals(6, schema1.getNumDataUnits()); + assertEquals(3, schema1.getNumParityUnits()); + assertEquals("RS", schema1.getCodecName()); + + ECSchema schema2 = schemas.get(1); + assertEquals("RSk10m4", schema2.getSchemaName()); + assertEquals(3, schema2.getOptions().size()); + assertEquals(10, schema2.getNumDataUnits()); + assertEquals(4, schema2.getNumParityUnits()); + assertEquals("RS", schema2.getCodecName()); + } +} \ No newline at end of file From 97378e4cd0902fcfe4d591c92b47fc292c032438 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 30 Mar 2015 10:23:09 -0700 Subject: [PATCH 033/212] HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for HDFS-7742) --- .../server/blockmanagement/TestBlockManager.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 a33a4e366f67d..3a1b19e24c3ec 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 @@ -557,11 +557,11 @@ public void testFavorDecomUntilHardLimit() throws Exception { assertNotNull("Chooses decommissioning source node for a normal replication" + " if all available source nodes have reached their replication" + " limits below the hard limit.", - bm.chooseSourceDatanode( - aBlock, + bm.chooseSourceDatanodes( + bm.getStoredBlock(aBlock), cntNodes, liveNodes, - new NumberReplicas(), + new NumberReplicas(), new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)); @@ -571,11 +571,11 @@ public void testFavorDecomUntilHardLimit() throws Exception { assertNull("Does not choose a source decommissioning node for a normal" + " replication when all available nodes exceed the hard limit.", - bm.chooseSourceDatanode( - aBlock, + bm.chooseSourceDatanodes( + bm.getStoredBlock(aBlock), cntNodes, liveNodes, - new NumberReplicas(), + new NumberReplicas(), new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)); } From abf833a7b228fff2bca4f69cd9df99d532380038 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 30 Mar 2015 11:25:09 -0700 Subject: [PATCH 034/212] HDFS-7907. Erasure Coding: track invalid, corrupt, and under-recovery striped blocks in NameNode. Contributed by Jing Zhao. --- .../blockmanagement/BlockInfoStriped.java | 25 ++- .../server/blockmanagement/BlockManager.java | 206 +++++++++--------- .../blockmanagement/DecommissionManager.java | 85 ++++---- .../hdfs/server/namenode/FSNamesystem.java | 8 +- .../server/blockmanagement/TestNodeCount.java | 2 +- .../TestOverReplicatedBlocks.java | 4 +- 6 files changed, 175 insertions(+), 155 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 30b5ee715f122..4a85efbcbcd1d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -18,11 +18,13 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; + import java.io.DataOutput; import java.io.IOException; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; + /** * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. * @@ -37,7 +39,6 @@ * array to record the block index for each triplet. */ public class BlockInfoStriped extends BlockInfo { - private final int chunkSize = HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; private final short dataBlockNum; private final short parityBlockNum; /** @@ -132,6 +133,22 @@ int getStorageBlockIndex(DatanodeStorageInfo storage) { return i == -1 ? -1 : indices[i]; } + /** + * Identify the block stored in the given datanode storage. Note that + * the returned block has the same block Id with the one seen/reported by the + * DataNode. + */ + Block getBlockOnStorage(DatanodeStorageInfo storage) { + int index = getStorageBlockIndex(storage); + if (index < 0) { + return null; + } else { + Block block = new Block(this); + block.setBlockId(this.getBlockId() + index); + return block; + } + } + @Override boolean removeStorage(DatanodeStorageInfo storage) { int dnIndex = findStorageInfoFromEnd(storage); @@ -186,8 +203,8 @@ public long spaceConsumed() { // In case striped blocks, total usage by this striped blocks should // be the total of data blocks and parity blocks because // `getNumBytes` is the total of actual data block size. - return ((getNumBytes() - 1) / (dataBlockNum * chunkSize) + 1) - * chunkSize * parityBlockNum + getNumBytes(); + return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CHUNK_SIZE) + 1) + * BLOCK_STRIPED_CHUNK_SIZE * parityBlockNum + getNumBytes(); } @Override 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 363e687a3689e..b943ba4221f08 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 @@ -179,7 +179,11 @@ public int getPendingDataNodeMessageCount() { /** Store blocks -> datanodedescriptor(s) map of corrupt replicas */ final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap(); - /** Blocks to be invalidated. */ + /** + * Blocks to be invalidated. + * For a striped block to invalidate, we should track its individual internal + * blocks. + */ private final InvalidateBlocks invalidateBlocks; /** @@ -195,8 +199,8 @@ public int getPendingDataNodeMessageCount() { * Maps a StorageID to the set of blocks that are "extra" for this * DataNode. We'll eventually remove these extras. */ - public final Map> excessReplicateMap = - new TreeMap>(); + public final Map> excessReplicateMap = + new TreeMap<>(); /** * Store set of Blocks that need to be replicated 1 or more times. @@ -594,11 +598,11 @@ public short getMinStorageNum(BlockInfo block) { ((BlockInfoStriped) block).getDataBlockNum() : minReplication; } - public boolean checkMinStorage(BlockInfo block) { + public boolean hasMinStorage(BlockInfo block) { return countNodes(block).liveReplicas() >= getMinStorageNum(block); } - public boolean checkMinStorage(BlockInfo block, int liveNum) { + public boolean hasMinStorage(BlockInfo block, int liveNum) { return liveNum >= getMinStorageNum(block); } @@ -643,7 +647,7 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, return false; // already completed (e.g. by syncBlock) final boolean b = commitBlock(lastBlock, commitBlock); - if (checkMinStorage(lastBlock)) { + if (hasMinStorage(lastBlock)) { completeBlock(bc, bc.numBlocks() - 1, false); } return b; @@ -667,7 +671,7 @@ private BlockInfo completeBlock(final BlockCollection bc, } int numNodes = curBlock.numNodes(); - if (!force && !checkMinStorage(curBlock, numNodes)) { + if (!force && !hasMinStorage(curBlock, numNodes)) { throw new IOException("Cannot complete block: " + "block does not satisfy minimal replication requirement."); } @@ -765,7 +769,7 @@ public LocatedBlock convertLastBlockToUnderConstruction( // count in safe-mode. namesystem.adjustSafeModeBlockTotals( // decrement safe if we had enough - checkMinStorage(oldBlock, targets.length) ? -1 : 0, + hasMinStorage(oldBlock, targets.length) ? -1 : 0, // always decrement total blocks -1); @@ -1099,7 +1103,7 @@ private BlocksWithLocations getBlocksWithLocations(final DatanodeID datanode, /** Remove the blocks associated to the given datanode. */ void removeBlocksAssociatedTo(final DatanodeDescriptor node) { - final Iterator it = node.getBlockIterator(); + final Iterator it = node.getBlockIterator(); while(it.hasNext()) { removeStoredBlock(it.next(), node); } @@ -1113,10 +1117,10 @@ void removeBlocksAssociatedTo(final DatanodeDescriptor node) { /** Remove the blocks associated to the given DatanodeStorageInfo. */ void removeBlocksAssociatedTo(final DatanodeStorageInfo storageInfo) { assert namesystem.hasWriteLock(); - final Iterator it = storageInfo.getBlockIterator(); + final Iterator it = storageInfo.getBlockIterator(); DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); while(it.hasNext()) { - Block block = it.next(); + BlockInfo block = it.next(); removeStoredBlock(block, node); invalidateBlocks.remove(node, block); } @@ -1138,21 +1142,32 @@ void addToInvalidates(final Block block, final DatanodeInfo datanode) { * Adds block to list of blocks which will be invalidated on all its * datanodes. */ - private void addToInvalidates(Block b) { + private void addToInvalidates(BlockInfo storedBlock) { if (!namesystem.isPopulatingReplQueues()) { return; } StringBuilder datanodes = new StringBuilder(); - for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) { + for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock, + State.NORMAL)) { final DatanodeDescriptor node = storage.getDatanodeDescriptor(); - invalidateBlocks.add(b, node, false); - datanodes.append(node).append(" "); + final Block b = getBlockToInvalidate(storedBlock, storage); + if (b != null) { + invalidateBlocks.add(b, node, false); + datanodes.append(node).append(" "); + } } if (datanodes.length() != 0) { - blockLog.info("BLOCK* addToInvalidates: {} {}", b, datanodes.toString()); + blockLog.info("BLOCK* addToInvalidates: {} {}", storedBlock, + datanodes.toString()); } } + private Block getBlockToInvalidate(BlockInfo storedBlock, + DatanodeStorageInfo storage) { + return storedBlock.isStriped() ? + ((BlockInfoStriped) storedBlock).getBlockOnStorage(storage) : storedBlock; + } + /** * Remove all block invalidation tasks under this datanode UUID; * used when a datanode registers with a new UUID and the old one @@ -1210,18 +1225,18 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, DatanodeStorageInfo storageInfo, DatanodeDescriptor node) throws IOException { - if (b.corrupted.isDeleted()) { + if (b.stored.isDeleted()) { blockLog.info("BLOCK markBlockAsCorrupt: {} cannot be marked as" + " corrupt as it does not belong to any file", b); addToInvalidates(b.corrupted, node); return; } short expectedReplicas = - b.corrupted.getBlockCollection().getPreferredBlockReplication(); + b.stored.getBlockCollection().getPreferredBlockReplication(); // Add replica to the data-node if it is not already there if (storageInfo != null) { - storageInfo.addBlock(b.stored, b.reportedBlock); + storageInfo.addBlock(b.stored, b.corrupted); } // Add this replica to corruptReplicas Map @@ -1231,8 +1246,10 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, NumberReplicas numberOfReplicas = countNodes(b.stored); boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= expectedReplicas; - boolean minReplicationSatisfied = checkMinStorage(b.stored, + + boolean minReplicationSatisfied = hasMinStorage(b.stored, numberOfReplicas.liveReplicas()); + boolean hasMoreCorruptReplicas = minReplicationSatisfied && (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) > expectedReplicas; @@ -1424,7 +1441,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { if (numEffectiveReplicas >= requiredReplication) { if ( (pendingReplications.getNumReplicas(block) > 0) || - (blockHasEnoughRacks(block)) ) { + (blockHasEnoughRacks(block, requiredReplication)) ) { neededReplications.remove(block, priority); // remove from neededReplications blockLog.info("BLOCK* Removing {} from neededReplications as" + " it has enough replicas", block); @@ -1507,7 +1524,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { if (numEffectiveReplicas >= requiredReplication) { if ( (pendingReplications.getNumReplicas(block) > 0) || - (blockHasEnoughRacks(block)) ) { + (blockHasEnoughRacks(block, requiredReplication)) ) { neededReplications.remove(block, priority); // remove from neededReplications rw.targets = null; blockLog.info("BLOCK* Removing {} from neededReplications as" + @@ -1517,7 +1534,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { } if ( (numReplicas.liveReplicas() >= requiredReplication) && - (!blockHasEnoughRacks(block)) ) { + (!blockHasEnoughRacks(block, requiredReplication)) ) { if (rw.srcNodes[0].getNetworkLocation().equals( targets[0].getDatanodeDescriptor().getNetworkLocation())) { //No use continuing, unless a new rack in this case @@ -1711,7 +1728,7 @@ DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block, getStorageBlockIndex(storage)); } final DatanodeDescriptor node = storage.getDatanodeDescriptor(); - LightWeightLinkedSet excessBlocks = + LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node.getDatanodeUuid()); int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) @@ -1847,39 +1864,32 @@ private static class BlockInfoToAdd { * list of blocks that should be considered corrupt due to a block report. */ private static class BlockToMarkCorrupt { - /** The corrupted block in a datanode. */ - final BlockInfo corrupted; + /** + * The corrupted block in a datanode. This is the one reported by the + * datanode. + */ + final Block corrupted; /** The corresponding block stored in the BlockManager. */ final BlockInfo stored; - /** The block reported from a datanode */ - final Block reportedBlock; /** The reason to mark corrupt. */ final String reason; /** The reason code to be stored */ final Reason reasonCode; - BlockToMarkCorrupt(Block reported, BlockInfo corrupted, - BlockInfo stored, String reason, Reason reasonCode) { - Preconditions.checkNotNull(reported, "reported is null"); + BlockToMarkCorrupt(Block corrupted, BlockInfo stored, String reason, + Reason reasonCode) { Preconditions.checkNotNull(corrupted, "corrupted is null"); Preconditions.checkNotNull(stored, "stored is null"); - this.reportedBlock = reported; this.corrupted = corrupted; this.stored = stored; this.reason = reason; this.reasonCode = reasonCode; } - BlockToMarkCorrupt(Block reported, BlockInfo stored, String reason, - Reason reasonCode) { - this(reported, stored, stored, reason, reasonCode); - } - - BlockToMarkCorrupt(Block reported, BlockInfo stored, long gs, + BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs, String reason, Reason reasonCode) { - this(reported, BlockInfo.copyOf(stored), stored, reason, - reasonCode); + this(corrupted, stored, reason, reasonCode); //the corrupted block in datanode has a different generation stamp corrupted.setGenerationStamp(gs); } @@ -2098,10 +2108,10 @@ private Collection processReport( // between the old and new block report. // Collection toAdd = new LinkedList<>(); - Collection toRemove = new TreeSet(); - Collection toInvalidate = new LinkedList(); - Collection toCorrupt = new LinkedList(); - Collection toUC = new LinkedList(); + Collection toRemove = new TreeSet<>(); + Collection toInvalidate = new LinkedList<>(); + Collection toCorrupt = new LinkedList<>(); + Collection toUC = new LinkedList<>(); reportDiff(storageInfo, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC); @@ -2110,7 +2120,7 @@ private Collection processReport( for (StatefulBlockInfo b : toUC) { addStoredBlockUnderConstruction(b, storageInfo); } - for (Block b : toRemove) { + for (BlockInfo b : toRemove) { removeStoredBlock(b, node); } int numBlocksLogged = 0; @@ -2250,7 +2260,7 @@ private void processFirstBlockReport( private void reportDiff(DatanodeStorageInfo storageInfo, BlockListAsLongs newReport, Collection toAdd, // add to DatanodeDescriptor - Collection toRemove, // remove from 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 @@ -2285,8 +2295,9 @@ private void reportDiff(DatanodeStorageInfo storageInfo, // collect blocks that have not been reported // all of them are next to the delimiter Iterator it = storageInfo.new BlockIterator(delimiter.getNext(0)); - while(it.hasNext()) + while (it.hasNext()) { toRemove.add(it.next()); + } storageInfo.removeBlock(delimiter); } @@ -2617,7 +2628,7 @@ private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported, // Now check for completion of blocks and safe block count int numCurrentReplica = countLiveNodes(storedBlock); if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED - && checkMinStorage(storedBlock, numCurrentReplica)) { + && hasMinStorage(storedBlock, numCurrentReplica)) { completeBlock(storedBlock.getBlockCollection(), storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block @@ -2692,7 +2703,7 @@ private Block addStoredBlock(final BlockInfo block, + pendingReplications.getNumReplicas(storedBlock); if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED && - checkMinStorage(storedBlock, numLiveReplicas)) { + hasMinStorage(storedBlock, numLiveReplicas)) { storedBlock = completeBlock(bc, storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block @@ -2730,7 +2741,7 @@ private Block addStoredBlock(final BlockInfo block, int numCorruptNodes = num.corruptReplicas(); if (numCorruptNodes != corruptReplicasCount) { LOG.warn("Inconsistent number of corrupt replicas for " + - storedBlock + "blockMap has " + numCorruptNodes + + storedBlock + ". blockMap has " + numCorruptNodes + " but corrupt replicas map has " + corruptReplicasCount); } if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) { @@ -3004,14 +3015,14 @@ public void setReplication(final short oldRepl, final short newRepl, * If there are any extras, call chooseExcessReplicates() to * mark them in the excessReplicateMap. */ - private void processOverReplicatedBlock(final Block block, + private void processOverReplicatedBlock(final BlockInfo block, final short replication, final DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) { assert namesystem.hasWriteLock(); if (addedNode == delNodeHint) { delNodeHint = null; } - Collection nonExcess = new ArrayList(); + Collection nonExcess = new ArrayList<>(); Collection corruptNodes = corruptReplicas .getNodes(block); for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) { @@ -3025,8 +3036,8 @@ private void processOverReplicatedBlock(final Block block, postponeBlock(block); return; } - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(cur - .getDatanodeUuid()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get( + cur.getDatanodeUuid()); if (excessBlocks == null || !excessBlocks.contains(block)) { if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { // exclude corrupt replicas @@ -3056,22 +3067,22 @@ private void processOverReplicatedBlock(final Block block, * then pick a node with least free space */ private void chooseExcessReplicates(final Collection nonExcess, - Block b, short replication, + BlockInfo storedBlock, short replication, DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint, BlockPlacementPolicy replicator) { assert namesystem.hasWriteLock(); // first form a rack to datanodes map and - BlockCollection bc = getBlockCollection(b); - final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID()); + BlockCollection bc = getBlockCollection(storedBlock); + final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( + bc.getStoragePolicyID()); final List excessTypes = storagePolicy.chooseExcess( replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); - final Map> rackMap - = new HashMap>(); - final List moreThanOne = new ArrayList(); - final List exactlyOne = new ArrayList(); + final Map> rackMap = new HashMap<>(); + final List moreThanOne = new ArrayList<>(); + final List exactlyOne = new ArrayList<>(); // split nodes into two sets // moreThanOne contains nodes on rack with more than one replica @@ -3092,7 +3103,7 @@ private void chooseExcessReplicates(final Collection nonExc moreThanOne, excessTypes)) { cur = delNodeHintStorage; } else { // regular excessive replica removal - cur = replicator.chooseReplicaToDelete(bc, b, replication, + cur = replicator.chooseReplicaToDelete(bc, storedBlock, replication, moreThanOne, exactlyOne, excessTypes); } firstOne = false; @@ -3102,7 +3113,7 @@ private void chooseExcessReplicates(final Collection nonExc exactlyOne, cur); nonExcess.remove(cur); - addToExcessReplicate(cur.getDatanodeDescriptor(), b); + addToExcessReplicate(cur.getDatanodeDescriptor(), storedBlock); // // The 'excessblocks' tracks blocks until we get confirmation @@ -3111,11 +3122,12 @@ private void chooseExcessReplicates(final Collection nonExc // // The 'invalidate' list is used to inform the datanode the block // should be deleted. Items are removed from the invalidate list - // upon giving instructions to the namenode. + // upon giving instructions to the datanodes. // - addToInvalidates(b, cur.getDatanodeDescriptor()); + final Block blockToInvalidate = getBlockToInvalidate(storedBlock, cur); + addToInvalidates(blockToInvalidate, cur.getDatanodeDescriptor()); blockLog.info("BLOCK* chooseExcessReplicates: " - +"({}, {}) is added to invalidated blocks set", cur, b); + +"({}, {}) is added to invalidated blocks set", cur, storedBlock); } } @@ -3140,17 +3152,18 @@ static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint, } } - private void addToExcessReplicate(DatanodeInfo dn, Block block) { + private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) { assert namesystem.hasWriteLock(); - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get( + dn.getDatanodeUuid()); if (excessBlocks == null) { - excessBlocks = new LightWeightLinkedSet(); + excessBlocks = new LightWeightLinkedSet<>(); excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks); } - if (excessBlocks.add(block)) { + if (excessBlocks.add(storedBlock)) { excessBlocksCount.incrementAndGet(); blockLog.debug("BLOCK* addToExcessReplicate: ({}, {}) is added to" - + " excessReplicateMap", dn, block); + + " excessReplicateMap", dn, storedBlock); } } @@ -3169,14 +3182,13 @@ private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block, * Modify (block-->datanode) map. Possibly generate replication tasks, if the * removed block is still valid. */ - public void removeStoredBlock(Block block, DatanodeDescriptor node) { - blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node); + public void removeStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) { + blockLog.debug("BLOCK* removeStoredBlock: {} from {}", storedBlock, node); assert (namesystem.hasWriteLock()); { - BlockInfo storedBlock = getStoredBlock(block); if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) { blockLog.debug("BLOCK* removeStoredBlock: {} has already been" + - " removed from node {}", block, node); + " removed from node {}", storedBlock, node); return; } @@ -3186,7 +3198,7 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { // necessary. In that case, put block on a possibly-will- // be-replicated list. // - BlockCollection bc = blocksMap.getBlockCollection(block); + BlockCollection bc = storedBlock.getBlockCollection(); if (bc != null) { namesystem.decrementSafeBlockCount(storedBlock); updateNeededReplications(storedBlock, -1, 0); @@ -3196,13 +3208,13 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { // We've removed a block from a node, so it's definitely no longer // in "excess" there. // - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node - .getDatanodeUuid()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get( + node.getDatanodeUuid()); if (excessBlocks != null) { - if (excessBlocks.remove(block)) { + if (excessBlocks.remove(storedBlock)) { excessBlocksCount.decrementAndGet(); blockLog.debug("BLOCK* removeStoredBlock: {} is removed from " + - "excessBlocks", block); + "excessBlocks", storedBlock); if (excessBlocks.size() == 0) { excessReplicateMap.remove(node.getDatanodeUuid()); } @@ -3210,7 +3222,7 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { } // Remove the replica from corruptReplicas - corruptReplicas.removeFromCorruptReplicasMap(block, node); + corruptReplicas.removeFromCorruptReplicasMap(storedBlock, node); } } @@ -3344,7 +3356,7 @@ public void processIncrementalBlockReport(final DatanodeID nodeID, for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) { switch (rdbi.getStatus()) { case DELETED_BLOCK: - removeStoredBlock(storageInfo, rdbi.getBlock(), node); + removeStoredBlock(storageInfo, getStoredBlock(rdbi.getBlock()), node); deleted++; break; case RECEIVED_BLOCK: @@ -3395,8 +3407,8 @@ public NumberReplicas countNodes(BlockInfo b) { } else if (node.isDecommissioned()) { decommissioned++; } else { - LightWeightLinkedSet blocksExcess = excessReplicateMap.get(node - .getDatanodeUuid()); + LightWeightLinkedSet blocksExcess = excessReplicateMap.get( + node.getDatanodeUuid()); if (blocksExcess != null && blocksExcess.contains(b)) { excess++; } else { @@ -3449,13 +3461,13 @@ void processOverReplicatedBlocksOnReCommission( int numOverReplicated = 0; while(it.hasNext()) { final BlockInfo block = it.next(); - BlockCollection bc = blocksMap.getBlockCollection(block); - short expectedReplication = bc.getPreferredBlockReplication(); + int expectedReplication = this.getReplication(block); NumberReplicas num = countNodes(block); int numCurrentReplica = num.liveReplicas(); if (numCurrentReplica > expectedReplication) { // over-replicated block - processOverReplicatedBlock(block, expectedReplication, null, null); + processOverReplicatedBlock(block, (short) expectedReplication, null, + null); numOverReplicated++; } } @@ -3655,21 +3667,20 @@ private int invalidateWorkForOneNode(DatanodeInfo dn) { return toInvalidate.size(); } - boolean blockHasEnoughRacks(Block b) { + // TODO: update the enough rack logic for striped blocks + boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) { if (!this.shouldCheckForEnoughRacks) { return true; } boolean enoughRacks = false; - Collection corruptNodes = - corruptReplicas.getNodes(b); - int numExpectedReplicas = getReplication(b); + Collection corruptNodes = + corruptReplicas.getNodes(storedBlock); String rackName = null; - for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { + for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) { final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { if ((corruptNodes == null ) || !corruptNodes.contains(cur)) { - if (numExpectedReplicas == 1 || - (numExpectedReplicas > 1 && + if (expectedStorageNum == 1 || (expectedStorageNum > 1 && !datanodeManager.hasClusterEverBeenMultiRack())) { enoughRacks = true; break; @@ -3691,8 +3702,8 @@ boolean blockHasEnoughRacks(Block b) { * A block needs replication if the number of replicas is less than expected * or if it does not have enough racks. */ - boolean isNeededReplication(Block b, int expected, int current) { - return current < expected || !blockHasEnoughRacks(b); + boolean isNeededReplication(BlockInfo storedBlock, int expected, int current) { + return current < expected || !blockHasEnoughRacks(storedBlock, expected); } public long getMissingBlocksCount() { @@ -3876,8 +3887,7 @@ public static LocatedBlock newLocatedBlock( /** * This class is used internally by {@link this#computeRecoveryWorkForBlocks} * to represent a task to recover a block through replication or erasure - * coding. Recovery is done by transferring data from {@link srcNodes} to - * {@link targets} + * coding. Recovery is done by transferring data from srcNodes to targets */ private static class BlockRecoveryWork { protected final BlockInfo block; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java index dc697f06436af..37ce8e34bc57e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java @@ -100,7 +100,7 @@ public class DecommissionManager { * reports or other events. Before being finally marking as decommissioned, * another check is done with the actual block map. */ - private final TreeMap> + private final TreeMap> decomNodeBlocks; /** @@ -244,12 +244,12 @@ private void setDecommissioned(DatanodeDescriptor dn) { } /** - * Checks whether a block is sufficiently replicated for decommissioning. - * Full-strength replication is not always necessary, hence "sufficient". + * Checks whether a block is sufficiently replicated/stored for + * decommissioning. For replicated blocks or striped blocks, full-strength + * replication or storage is not always necessary, hence "sufficient". * @return true if sufficient, else false. */ - private boolean isSufficientlyReplicated(BlockInfoContiguous block, - BlockCollection bc, + private boolean isSufficient(BlockInfo block, BlockCollection bc, NumberReplicas numberReplicas) { final int numExpected = bc.getPreferredBlockReplication(); final int numLive = numberReplicas.liveReplicas(); @@ -265,18 +265,19 @@ private boolean isSufficientlyReplicated(BlockInfoContiguous block, if (numExpected > numLive) { if (bc.isUnderConstruction() && block.equals(bc.getLastBlock())) { // Can decom a UC block as long as there will still be minReplicas - if (numLive >= blockManager.minReplication) { + if (blockManager.hasMinStorage(block, numLive)) { LOG.trace("UC block {} sufficiently-replicated since numLive ({}) " - + ">= minR ({})", block, numLive, blockManager.minReplication); + + ">= minR ({})", block, numLive, + blockManager.getMinStorageNum(block)); return true; } else { LOG.trace("UC block {} insufficiently-replicated since numLive " + "({}) < minR ({})", block, numLive, - blockManager.minReplication); + blockManager.getMinStorageNum(block)); } } else { // Can decom a non-UC as long as the default replication is met - if (numLive >= blockManager.defaultReplication) { + if (numLive >= blockManager.getDefaultStorageNum(block)) { return true; } } @@ -412,7 +413,7 @@ private void processPendingNodes() { } private void check() { - final Iterator>> + final Iterator>> it = new CyclicIteration<>(decomNodeBlocks, iterkey).iterator(); final LinkedList toRemove = new LinkedList<>(); @@ -420,10 +421,10 @@ private void check() { && !exceededNumBlocksPerCheck() && !exceededNumNodesPerCheck()) { numNodesChecked++; - final Map.Entry> + final Map.Entry> entry = it.next(); final DatanodeDescriptor dn = entry.getKey(); - AbstractList blocks = entry.getValue(); + AbstractList blocks = entry.getValue(); boolean fullScan = false; if (blocks == null) { // This is a newly added datanode, run through its list to schedule @@ -431,14 +432,14 @@ private void check() { // that are insufficiently replicated for further tracking LOG.debug("Newly-added node {}, doing full scan to find " + "insufficiently-replicated blocks.", dn); - blocks = handleInsufficientlyReplicated(dn); + blocks = handleInsufficientlyStored(dn); decomNodeBlocks.put(dn, blocks); fullScan = true; } else { // This is a known datanode, check if its # of insufficiently // replicated blocks has dropped to zero and if it can be decommed LOG.debug("Processing decommission-in-progress node {}", dn); - pruneSufficientlyReplicated(dn, blocks); + pruneReliableBlocks(dn, blocks); } if (blocks.size() == 0) { if (!fullScan) { @@ -450,7 +451,7 @@ private void check() { // marking the datanode as decommissioned LOG.debug("Node {} has finished replicating current set of " + "blocks, checking with the full block map.", dn); - blocks = handleInsufficientlyReplicated(dn); + blocks = handleInsufficientlyStored(dn); decomNodeBlocks.put(dn, blocks); } // If the full scan is clean AND the node liveness is okay, @@ -491,27 +492,25 @@ private void check() { } /** - * Removes sufficiently replicated blocks from the block list of a - * datanode. + * Removes reliable blocks from the block list of a datanode. */ - private void pruneSufficientlyReplicated(final DatanodeDescriptor datanode, - AbstractList blocks) { + private void pruneReliableBlocks(final DatanodeDescriptor datanode, + AbstractList blocks) { processBlocksForDecomInternal(datanode, blocks.iterator(), null, true); } /** - * Returns a list of blocks on a datanode that are insufficiently - * replicated, i.e. are under-replicated enough to prevent decommission. + * Returns a list of blocks on a datanode that are insufficiently replicated + * or require recovery, i.e. requiring recovery and should prevent + * decommission. *

- * As part of this, it also schedules replication work for - * any under-replicated blocks. + * As part of this, it also schedules replication/recovery work. * - * @param datanode - * @return List of insufficiently replicated blocks + * @return List of blocks requiring recovery */ - private AbstractList handleInsufficientlyReplicated( + private AbstractList handleInsufficientlyStored( final DatanodeDescriptor datanode) { - AbstractList insufficient = new ChunkedArrayList<>(); + AbstractList insufficient = new ChunkedArrayList<>(); processBlocksForDecomInternal(datanode, datanode.getBlockIterator(), insufficient, false); return insufficient; @@ -520,24 +519,22 @@ private AbstractList handleInsufficientlyReplicated( /** * Used while checking if decommission-in-progress datanodes can be marked * as decommissioned. Combines shared logic of - * pruneSufficientlyReplicated and handleInsufficientlyReplicated. + * pruneReliableBlocks and handleInsufficientlyStored. * * @param datanode Datanode * @param it Iterator over the blocks on the * datanode - * @param insufficientlyReplicated Return parameter. If it's not null, + * @param insufficientList Return parameter. If it's not null, * will contain the insufficiently * replicated-blocks from the list. - * @param pruneSufficientlyReplicated whether to remove sufficiently - * replicated blocks from the iterator - * @return true if there are under-replicated blocks in the provided block - * iterator, else false. + * @param pruneReliableBlocks whether to remove blocks reliable + * enough from the iterator */ private void processBlocksForDecomInternal( final DatanodeDescriptor datanode, - final Iterator it, - final List insufficientlyReplicated, - boolean pruneSufficientlyReplicated) { + final Iterator it, + final List insufficientList, + boolean pruneReliableBlocks) { boolean firstReplicationLog = true; int underReplicatedBlocks = 0; int decommissionOnlyReplicas = 0; @@ -552,7 +549,7 @@ private void processBlocksForDecomInternal( it.remove(); continue; } - BlockCollection bc = blockManager.blocksMap.getBlockCollection(block); + BlockCollection bc = blockManager.getBlockCollection(block); if (bc == null) { // Orphan block, will be invalidated eventually. Skip. continue; @@ -560,7 +557,6 @@ private void processBlocksForDecomInternal( final NumberReplicas num = blockManager.countNodes(block); final int liveReplicas = num.liveReplicas(); - final int curReplicas = liveReplicas; // Schedule under-replicated blocks for replication if not already // pending @@ -571,7 +567,7 @@ private void processBlocksForDecomInternal( namesystem.isPopulatingReplQueues()) { // Process these blocks only when active NN is out of safe mode. blockManager.neededReplications.add(block, - curReplicas, + liveReplicas, num.decommissionedAndDecommissioning(), bc.getPreferredBlockReplication()); } @@ -579,17 +575,16 @@ private void processBlocksForDecomInternal( // Even if the block is under-replicated, // it doesn't block decommission if it's sufficiently replicated - BlockInfoContiguous blk = (BlockInfoContiguous) block; - if (isSufficientlyReplicated(blk, bc, num)) { - if (pruneSufficientlyReplicated) { + if (isSufficient(block, bc, num)) { + if (pruneReliableBlocks) { it.remove(); } continue; } // We've found an insufficiently replicated block. - if (insufficientlyReplicated != null) { - insufficientlyReplicated.add(blk); + if (insufficientList != null) { + insufficientList.add(block); } // Log if this is our first time through if (firstReplicationLog) { @@ -602,7 +597,7 @@ private void processBlocksForDecomInternal( if (bc.isUnderConstruction()) { underReplicatedInOpenFiles++; } - if ((curReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) { + if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) { decommissionOnlyReplicas++; } } 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 0460f5ce0a2f0..5fac43cc25b32 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 @@ -3273,7 +3273,7 @@ void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) { if (trackBlockCounts) { if (b.isComplete()) { numRemovedComplete++; - if (blockManager.checkMinStorage(b, b.numNodes())) { + if (blockManager.hasMinStorage(b, b.numNodes())) { numRemovedSafe++; } } @@ -3502,7 +3502,7 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, curBlock = blocks[nrCompleteBlocks]; if(!curBlock.isComplete()) break; - assert blockManager.checkMinStorage(curBlock) : + assert blockManager.hasMinStorage(curBlock) : "A COMPLETE block is not minimally replicated in " + src; } @@ -3538,7 +3538,7 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, // If penultimate block doesn't exist then its minReplication is met boolean penultimateBlockMinStorage = penultimateBlock == null || - blockManager.checkMinStorage(penultimateBlock); + blockManager.hasMinStorage(penultimateBlock); switch(lastBlockState) { case COMPLETE: @@ -3547,7 +3547,7 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, case COMMITTED: // Close file if committed blocks are minimally replicated if(penultimateBlockMinStorage && - blockManager.checkMinStorage(lastBlock)) { + blockManager.hasMinStorage(lastBlock)) { finalizeINodeFileUnderConstruction(src, pendingFile, iip.getLatestSnapshotId()); NameNode.stateChangeLog.warn("BLOCK*" 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 1c3f075d5f426..c33667d5e00f5 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 @@ -100,7 +100,7 @@ public void testNodeCount() throws Exception { DatanodeDescriptor nonExcessDN = null; for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) { final DatanodeDescriptor dn = storage.getDatanodeDescriptor(); - Collection blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid()); + 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 2d7bb440d0cd8..83b3aa0f6a178 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 @@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; -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.datanode.DataNode; @@ -42,7 +41,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; -import org.apache.hadoop.util.Time; import org.junit.Test; public class TestOverReplicatedBlocks { @@ -185,7 +183,7 @@ public void testChooseReplicaToDelete() throws Exception { // All replicas for deletion should be scheduled on lastDN. // And should not actually be deleted, because lastDN does not heartbeat. namesystem.readLock(); - Collection dnBlocks = + Collection dnBlocks = namesystem.getBlockManager().excessReplicateMap.get(lastDNid); assertEquals("Replicas on node " + lastDNid + " should have been deleted", SMALL_FILE_LENGTH / SMALL_BLOCK_SIZE, dnBlocks.size()); From cd655ee817a3307bc1a1a119eb4266978ecd7fb2 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 30 Mar 2015 13:35:36 -0700 Subject: [PATCH 035/212] HDFS-8005. Erasure Coding: simplify striped block recovery work computation and add tests. Contributed by Jing Zhao. --- .../server/blockmanagement/BlockManager.java | 134 +++++------ .../blockmanagement/DatanodeDescriptor.java | 14 +- .../hdfs/server/namenode/INodeFile.java | 1 + .../blockmanagement/TestBlockManager.java | 33 +-- .../TestRecoverStripedBlocks.java | 107 --------- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../namenode/TestRecoverStripedBlocks.java | 210 ++++++++++++++++++ 7 files changed, 290 insertions(+), 211 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java 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 b943ba4221f08..97d8379f5c354 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 @@ -541,7 +541,7 @@ private void dumpBlockMeta(Block block, PrintWriter out) { // source node returned is not used chooseSourceDatanodes(getStoredBlock(block), containingNodes, containingLiveReplicasNodes, numReplicas, - new LinkedList(), 1, UnderReplicatedBlocks.LEVEL); + new LinkedList(), UnderReplicatedBlocks.LEVEL); // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are // not included in the numReplicas.liveReplicas() count @@ -1389,7 +1389,7 @@ int computeBlockRecoveryWork(int blocksToProcess) { int computeRecoveryWorkForBlocks(List> blocksToRecover) { int requiredReplication, numEffectiveReplicas; List containingNodes; - BlockCollection bc = null; + BlockCollection bc; int additionalReplRequired; int scheduledWork = 0; @@ -1417,13 +1417,10 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { containingNodes = new ArrayList<>(); List liveReplicaNodes = new ArrayList<>(); NumberReplicas numReplicas = new NumberReplicas(); - List missingBlockIndices = new LinkedList<>(); - DatanodeDescriptor[] srcNodes; - int numSourceNodes = bc.isStriped() ? - HdfsConstants.NUM_DATA_BLOCKS : 1; - srcNodes = chooseSourceDatanodes( - block, containingNodes, liveReplicaNodes, numReplicas, - missingBlockIndices, numSourceNodes, priority); + List liveBlockIndices = new ArrayList<>(); + final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block, + containingNodes, liveReplicaNodes, numReplicas, + liveBlockIndices, priority); if(srcNodes == null || srcNodes.length == 0) { // block can not be replicated from any node LOG.debug("Block " + block + " cannot be recovered " + @@ -1455,15 +1452,14 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { } else { additionalReplRequired = 1; // Needed on a new rack } - if (bc.isStriped()) { + if (block.isStriped()) { + short[] indices = new short[liveBlockIndices.size()]; + for (int i = 0 ; i < liveBlockIndices.size(); i++) { + indices[i] = liveBlockIndices.get(i); + } ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes, containingNodes, liveReplicaNodes, additionalReplRequired, - priority); - short[] missingBlockArray = new short[missingBlockIndices.size()]; - for (int i = 0 ; i < missingBlockIndices.size(); i++) { - missingBlockArray[i] = missingBlockIndices.get(i); - } - ecw.setMissingBlockIndices(missingBlockArray); + priority, indices); recovWork.add(ecw); } else { recovWork.add(new ReplicationWork(block, bc, srcNodes, @@ -1543,15 +1539,14 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { } // Add block to the to be replicated list - if (bc.isStriped()) { + if (block.isStriped()) { assert rw instanceof ErasureCodingWork; assert rw.targets.length > 0; rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded( new ExtendedBlock(namesystem.getBlockPoolId(), block), rw.srcNodes, rw.targets, - ((ErasureCodingWork)rw).getMissingBlockIndicies()); - } - else { + ((ErasureCodingWork) rw).liveBlockIndicies); + } else { rw.srcNodes[0].addBlockToBeReplicated(block, targets); } scheduledWork++; @@ -1581,9 +1576,9 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { DatanodeStorageInfo[] targets = rw.targets; if (targets != null && targets.length != 0) { StringBuilder targetList = new StringBuilder("datanode(s)"); - for (int k = 0; k < targets.length; k++) { + for (DatanodeStorageInfo target : targets) { targetList.append(' '); - targetList.append(targets[k].getDatanodeDescriptor()); + targetList.append(target.getDatanodeDescriptor()); } blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes, rw.block, targetList); @@ -1694,11 +1689,8 @@ List getDatanodeDescriptors(List nodes) { * @param numReplicas NumberReplicas instance to be initialized with the * counts of live, corrupt, excess, and decommissioned * replicas of the given block. - * @param missingBlockIndices List to be populated with indices of missing - * blocks in a striped block group or missing - * replicas of a replicated block - * @param numSourceNodes integer specifying the number of source nodes to - * choose + * @param liveBlockIndices List to be populated with indices of healthy + * blocks in a striped block group * @param priority integer representing replication priority of the given * block * @return the array of DatanodeDescriptor of the chosen nodes from which to @@ -1709,24 +1701,20 @@ DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block, List containingNodes, List nodesContainingLiveReplicas, NumberReplicas numReplicas, - List missingBlockIndices, int numSourceNodes, int priority) { + List liveBlockIndices, int priority) { containingNodes.clear(); nodesContainingLiveReplicas.clear(); - LinkedList srcNodes = new LinkedList<>(); + List srcNodes = new ArrayList<>(); int live = 0; int decommissioned = 0; int decommissioning = 0; int corrupt = 0; int excess = 0; - missingBlockIndices.clear(); - Set healthyIndices = new HashSet<>(); + liveBlockIndices.clear(); + final boolean isStriped = block.isStriped(); Collection nodesCorrupt = corruptReplicas.getNodes(block); - for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { - if (block.isStriped()) { - healthyIndices.add((short) ((BlockInfoStriped) block). - getStorageBlockIndex(storage)); - } + for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) { final DatanodeDescriptor node = storage.getDatanodeDescriptor(); LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node.getDatanodeUuid()); @@ -1765,27 +1753,19 @@ else if (node.isDecommissionInProgress()) { if(node.isDecommissioned()) continue; - // We got this far, current node is a reasonable choice - if(srcNodes.size() < numSourceNodes) { + if(isStriped || srcNodes.isEmpty()) { srcNodes.add(node); + if (isStriped) { + liveBlockIndices.add((short) ((BlockInfoStriped) block). + getStorageBlockIndex(storage)); + } continue; } - // switch to a different node randomly + // for replicated block, switch to a different node randomly // this to prevent from deterministically selecting the same node even // if the node failed to replicate the block on previous iterations - if(ThreadLocalRandom.current().nextBoolean()) { - int pos = ThreadLocalRandom.current().nextInt(numSourceNodes); - if(!srcNodes.get(pos).isDecommissionInProgress()) { - srcNodes.set(pos, node); - } - } - } - if (block.isStriped()) { - for (short i = 0; i < HdfsConstants.NUM_DATA_BLOCKS + - HdfsConstants.NUM_PARITY_BLOCKS; i++) { - if (!healthyIndices.contains(i)) { - missingBlockIndices.add(i); - } + if (!isStriped && ThreadLocalRandom.current().nextBoolean()) { + srcNodes.set(0, node); } } if(numReplicas != null) @@ -3889,25 +3869,25 @@ public static LocatedBlock newLocatedBlock( * to represent a task to recover a block through replication or erasure * coding. Recovery is done by transferring data from srcNodes to targets */ - private static class BlockRecoveryWork { - protected final BlockInfo block; - protected final BlockCollection bc; + private abstract static class BlockRecoveryWork { + final BlockInfo block; + final BlockCollection bc; /** * An erasure coding recovery task has multiple source nodes. * A replication task only has 1 source node, stored on top of the array */ - protected final DatanodeDescriptor[] srcNodes; + final DatanodeDescriptor[] srcNodes; /** Nodes containing the block; avoid them in choosing new targets */ - protected final List containingNodes; + final List containingNodes; /** Required by {@link BlockPlacementPolicy#chooseTarget} */ - protected final List liveReplicaStorages; - protected final int additionalReplRequired; + final List liveReplicaStorages; + final int additionalReplRequired; - protected DatanodeStorageInfo[] targets; - protected final int priority; + DatanodeStorageInfo[] targets; + final int priority; - public BlockRecoveryWork(BlockInfo block, + BlockRecoveryWork(BlockInfo block, BlockCollection bc, DatanodeDescriptor[] srcNodes, List containingNodes, @@ -3924,15 +3904,13 @@ public BlockRecoveryWork(BlockInfo block, this.targets = null; } - protected void chooseTargets(BlockPlacementPolicy blockplacement, + abstract void chooseTargets(BlockPlacementPolicy blockplacement, BlockStoragePolicySuite storagePolicySuite, - Set excludedNodes) { - } + Set excludedNodes); } private static class ReplicationWork extends BlockRecoveryWork { - - public ReplicationWork(BlockInfo block, + ReplicationWork(BlockInfo block, BlockCollection bc, DatanodeDescriptor[] srcNodes, List containingNodes, @@ -3944,7 +3922,8 @@ public ReplicationWork(BlockInfo block, LOG.debug("Creating a ReplicationWork to recover " + block); } - protected void chooseTargets(BlockPlacementPolicy blockplacement, + @Override + void chooseTargets(BlockPlacementPolicy blockplacement, BlockStoragePolicySuite storagePolicySuite, Set excludedNodes) { assert srcNodes.length > 0 @@ -3961,30 +3940,23 @@ protected void chooseTargets(BlockPlacementPolicy blockplacement, } private static class ErasureCodingWork extends BlockRecoveryWork { + final short[] liveBlockIndicies; - private short[] missingBlockIndicies = null; - - public ErasureCodingWork(BlockInfo block, + ErasureCodingWork(BlockInfo block, BlockCollection bc, DatanodeDescriptor[] srcNodes, List containingNodes, List liveReplicaStorages, int additionalReplRequired, - int priority) { + int priority, short[] liveBlockIndicies) { super(block, bc, srcNodes, containingNodes, liveReplicaStorages, additionalReplRequired, priority); + this.liveBlockIndicies = liveBlockIndicies; LOG.debug("Creating an ErasureCodingWork to recover " + block); } - public short[] getMissingBlockIndicies() { - return missingBlockIndicies; - } - - public void setMissingBlockIndices(short[] missingBlockIndicies) { - this.missingBlockIndicies = missingBlockIndicies; - } - - protected void chooseTargets(BlockPlacementPolicy blockplacement, + @Override + void chooseTargets(BlockPlacementPolicy blockplacement, BlockStoragePolicySuite storagePolicySuite, Set excludedNodes) { try { 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 7bc5e7e48a139..15427f7590632 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 @@ -106,14 +106,14 @@ public static class BlockECRecoveryInfo { public final ExtendedBlock block; public final DatanodeDescriptor[] sources; public final DatanodeStorageInfo[] targets; - public final short[] missingBlockIndices; + public final short[] liveBlockIndices; BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources, - DatanodeStorageInfo[] targets, short[] missingBlockIndices) { + DatanodeStorageInfo[] targets, short[] liveBlockIndices) { this.block = block; this.sources = sources; this.targets = targets; - this.missingBlockIndices = missingBlockIndices; + this.liveBlockIndices = liveBlockIndices; } @Override @@ -122,6 +122,7 @@ public String toString() { append("Recovering ").append(block). append(" From: ").append(Arrays.asList(sources)). append(" To: ").append(Arrays.asList(targets)).append(")\n"). + append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)). toString(); } } @@ -635,10 +636,10 @@ void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { * Store block erasure coding work. */ void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources, - DatanodeStorageInfo[] targets, short[] missingBlockIndicies) { + DatanodeStorageInfo[] targets, short[] liveBlockIndices) { assert(block != null && sources != null && sources.length > 0); BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets, - missingBlockIndicies); + liveBlockIndices); erasurecodeBlocks.offer(task); BlockManager.LOG.debug("Adding block recovery task " + task + "to " + getName() + ", current queue size is " + @@ -679,7 +680,8 @@ int getNumberOfBlocksToBeReplicated() { /** * The number of work items that are pending to be replicated */ - int getNumberOfBlocksToBeErasureCoded() { + @VisibleForTesting + public int getNumberOfBlocksToBeErasureCoded() { return erasurecodeBlocks.size(); } 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 d5a809c8fdec7..51e24db657e92 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 @@ -404,6 +404,7 @@ public final short getFileReplication(int snapshot) { /** The same as getFileReplication(null). */ @Override // INodeFileAttributes + // TODO striped public final short getFileReplication() { return getFileReplication(CURRENT_STATE_ID); } 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 3a1b19e24c3ec..074be16e20ba2 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 @@ -513,30 +513,33 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception { cntNodes, liveNodes, new NumberReplicas(), - new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]); + new ArrayList(), + UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]); - assertNull("Does not choose a source node for a less-than-highest-priority" - + " replication since all available source nodes have reached" - + " their replication limits.", + assertEquals("Does not choose a source node for a less-than-highest-priority" + + " replication since all available source nodes have reached" + + " their replication limits.", 0, bm.chooseSourceDatanodes( bm.getStoredBlock(aBlock), cntNodes, liveNodes, new NumberReplicas(), - new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)[0]); + new ArrayList(), + UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED).length); // Increase the replication count to test replication count > hard limit DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] }; origNodes.get(0).addBlockToBeReplicated(aBlock, targets); - assertNull("Does not choose a source node for a highest-priority" - + " replication when all available nodes exceed the hard limit.", + assertEquals("Does not choose a source node for a highest-priority" + + " replication when all available nodes exceed the hard limit.", 0, bm.chooseSourceDatanodes( bm.getStoredBlock(aBlock), cntNodes, liveNodes, new NumberReplicas(), - new LinkedList(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]); + new ArrayList(), + UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).length); } @Test @@ -561,26 +564,24 @@ public void testFavorDecomUntilHardLimit() throws Exception { bm.getStoredBlock(aBlock), cntNodes, liveNodes, - new NumberReplicas(), new LinkedList(), 1, - UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)); + new NumberReplicas(), new LinkedList(), + UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)[0]); // Increase the replication count to test replication count > hard limit DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] }; origNodes.get(0).addBlockToBeReplicated(aBlock, targets); - assertNull("Does not choose a source decommissioning node for a normal" - + " replication when all available nodes exceed the hard limit.", + assertEquals("Does not choose a source decommissioning node for a normal" + + " replication when all available nodes exceed the hard limit.", 0, bm.chooseSourceDatanodes( bm.getStoredBlock(aBlock), cntNodes, liveNodes, - new NumberReplicas(), new LinkedList(), 1, - UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)); + new NumberReplicas(), new LinkedList(), + UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).length); } - - @Test public void testSafeModeIBR() throws Exception { DatanodeDescriptor node = spy(nodes.get(0)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java deleted file mode 100644 index d883c9b3234fd..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java +++ /dev/null @@ -1,107 +0,0 @@ -/** - * 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 org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; -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.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.client.HdfsAdmin; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.Iterator; - -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME; -import static org.junit.Assert.assertTrue; - -public class TestRecoverStripedBlocks { - private final short GROUP_SIZE = - HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; - private final short NUM_OF_DATANODES = GROUP_SIZE + 1; - private Configuration conf; - private MiniDFSCluster cluster; - private DistributedFileSystem fs; - private static final int BLOCK_SIZE = 1024; - private HdfsAdmin dfsAdmin; - private FSNamesystem namesystem; - private Path ECFilePath; - - @Before - public void setupCluster() throws IOException { - conf = new HdfsConfiguration(); - conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); - // Large value to make sure the pending replication request can stay in - // DatanodeDescriptor.replicateBlocks before test timeout. - conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100); - // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via - // chooseUnderReplicatedBlocks at once. - conf.setInt( - DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5); - - cluster = new MiniDFSCluster.Builder(conf). - numDataNodes(NUM_OF_DATANODES).build(); - cluster.waitActive(); - fs = cluster.getFileSystem(); - dfsAdmin = new HdfsAdmin(cluster.getURI(), conf); - namesystem = cluster.getNamesystem(); - ECFilePath = new Path("/ecfile"); - DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, GROUP_SIZE, 0); - dfsAdmin.setStoragePolicy(ECFilePath, EC_STORAGE_POLICY_NAME); - } - - @Test - public void testMissingStripedBlock() throws Exception { - final BlockManager bm = cluster.getNamesystem().getBlockManager(); - ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, ECFilePath); - Iterator storageInfos = - bm.blocksMap.getStorages(b.getLocalBlock()) - .iterator(); - - DatanodeDescriptor firstDn = storageInfos.next().getDatanodeDescriptor(); - Iterator it = firstDn.getBlockIterator(); - int missingBlkCnt = 0; - while (it.hasNext()) { - BlockInfo blk = it.next(); - BlockManager.LOG.debug("Block " + blk + " will be lost"); - missingBlkCnt++; - } - BlockManager.LOG.debug("Missing in total " + missingBlkCnt + " blocks"); - - bm.getDatanodeManager().removeDatanode(firstDn); - - bm.computeDatanodeWork(); - - short cnt = 0; - for (DataNode dn : cluster.getDataNodes()) { - DatanodeDescriptor dnDescriptor = - bm.getDatanodeManager().getDatanode(dn.getDatanodeUuid()); - cnt += dnDescriptor.getNumberOfBlocksToBeErasureCoded(); - } - - assertTrue("Counting the number of outstanding EC tasks", cnt == missingBlkCnt); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 7d7c81e0fe405..215a4e4b8efb1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -224,7 +224,7 @@ public void testAddUCReplica() throws Exception { int i = 0; for (DataNode dn : cluster.getDataNodes()) { final Block block = new Block(lastBlock.getBlockId() + i++, - lastBlock.getGenerationStamp(), 0); + 0, lastBlock.getGenerationStamp()); DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); storageIDs.add(storage.getStorageID()); StorageReceivedDeletedBlocks[] reports = DFSTestUtil diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java new file mode 100644 index 0000000000000..b9fd4fee16e72 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -0,0 +1,210 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +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.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.client.HdfsAdmin; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +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.DatanodeDescriptor.BlockECRecoveryInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; + +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestRecoverStripedBlocks { + private final short GROUP_SIZE = + NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; + private MiniDFSCluster cluster; + private final Path dirPath = new Path("/dir"); + private Path filePath = new Path(dirPath, "file"); + + @Before + public void setup() throws IOException { + final Configuration conf = new HdfsConfiguration(); + // Large value to make sure the pending replication request can stay in + // DatanodeDescriptor.replicateBlocks before test timeout. + conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100); + // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via + // chooseUnderReplicatedBlocks at once. + conf.setInt( + DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5); + + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE + 1) + .build(); + cluster.waitActive(); + } + + @After + public void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + + public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, + int numBlocks) throws Exception { + DistributedFileSystem dfs = cluster.getFileSystem(); + dfs.mkdirs(dir); + dfs.setStoragePolicy(dir, EC_STORAGE_POLICY_NAME); + + FSDataOutputStream out = null; + try { + out = dfs.create(file, (short) 1); // create an empty file + + FSNamesystem ns = cluster.getNamesystem(); + FSDirectory fsdir = ns.getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + + ExtendedBlock previous = null; + for (int i = 0; i < numBlocks; i++) { + Block newBlock = createBlock(cluster.getDataNodes(), ns, + file.toString(), fileNode, dfs.getClient().getClientName(), + previous); + previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock); + } + + ns.completeFile(file.toString(), dfs.getClient().getClientName(), + previous, fileNode.getId()); + } finally { + IOUtils.cleanup(null, out); + } + } + + static Block createBlock(List dataNodes, FSNamesystem ns, + String file, INodeFile fileNode, String clientName, + ExtendedBlock previous) throws Exception { + ns.getAdditionalBlock(file, fileNode.getId(), clientName, previous, null, + null); + + final BlockInfo lastBlock = fileNode.getLastBlock(); + final int groupSize = fileNode.getBlockReplication(); + // 1. RECEIVING_BLOCK IBR + int i = 0; + for (DataNode dn : dataNodes) { + if (i < groupSize) { + final Block block = new Block(lastBlock.getBlockId() + i++, 0, + lastBlock.getGenerationStamp()); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage); + for (StorageReceivedDeletedBlocks report : reports) { + ns.processIncrementalBlockReport(dn.getDatanodeId(), report); + } + } + } + + // 2. RECEIVED_BLOCK IBR + i = 0; + for (DataNode dn : dataNodes) { + if (i < groupSize) { + final Block block = new Block(lastBlock.getBlockId() + i++, + BLOCK_STRIPED_CHUNK_SIZE, lastBlock.getGenerationStamp()); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + for (StorageReceivedDeletedBlocks report : reports) { + ns.processIncrementalBlockReport(dn.getDatanodeId(), report); + } + } + } + + lastBlock.setNumBytes(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS); + return lastBlock; + } + + @Test + public void testMissingStripedBlock() throws Exception { + final int numBlocks = 4; + createECFile(cluster, filePath, dirPath, numBlocks); + + // make sure the file is complete in NN + final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() + .getINode4Write(filePath.toString()).asFile(); + assertFalse(fileNode.isUnderConstruction()); + assertTrue(fileNode.isWithStripedBlocks()); + BlockInfo[] blocks = fileNode.getBlocks(); + assertEquals(numBlocks, blocks.length); + for (BlockInfo blk : blocks) { + assertTrue(blk.isStriped()); + assertTrue(blk.isComplete()); + assertEquals(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes()); + final BlockInfoStriped sb = (BlockInfoStriped) blk; + assertEquals(GROUP_SIZE, sb.numNodes()); + } + + final BlockManager bm = cluster.getNamesystem().getBlockManager(); + BlockInfo firstBlock = fileNode.getBlocks()[0]; + DatanodeStorageInfo[] storageInfos = bm.getStorages(firstBlock); + + DatanodeDescriptor secondDn = storageInfos[1].getDatanodeDescriptor(); + assertEquals(numBlocks, secondDn.numBlocks()); + + bm.getDatanodeManager().removeDatanode(secondDn); + + BlockManagerTestUtil.getComputedDatanodeWork(bm); + + // all the recovery work will be scheduled on the last DN + DataNode lastDn = cluster.getDataNodes().get(GROUP_SIZE); + DatanodeDescriptor last = + bm.getDatanodeManager().getDatanode(lastDn.getDatanodeId()); + assertEquals("Counting the number of outstanding EC tasks", numBlocks, + last.getNumberOfBlocksToBeErasureCoded()); + List recovery = last.getErasureCodeCommand(numBlocks); + for (BlockECRecoveryInfo info : recovery) { + assertEquals(1, info.targets.length); + assertEquals(last, info.targets[0].getDatanodeDescriptor()); + assertEquals(GROUP_SIZE - 1, info.sources.length); + assertEquals(GROUP_SIZE - 1, info.liveBlockIndices.length); + } + } +} From af8eaacdb40afba327b776e2b4f374be624289fc Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 31 Mar 2015 15:12:09 +0530 Subject: [PATCH 036/212] HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 40 ++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 21e4c033da679..a686315142570 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -3,6 +3,44 @@ HDFS-7347. Configurable erasure coding policy for individual files and directories ( Zhe Zhang via vinayakumarb ) - HDFS-7716. Add a test for BlockGroup support in FSImage. + HDFS-7339. Representing striped block groups in NameNode with hierarchical + naming protocol ( Zhe Zhang ) + + HDFS-7652. Process block reports for erasure coded blocks (Zhe Zhang) + + HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info (Jing Zhao) + + HDFS-7749. Erasure Coding: Add striped block support in INodeFile (Jing Zhao) + + HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode + (Jing Zhao via Zhe Zhang) + + HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print + striped blocks (Takuya Fukudome via jing9) + + HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from + striped files (Jing Zhao) + + HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped + blocks ( Kai Sasaki via jing9 ) + + HDFS-7912. Erasure Coding: track BlockInfo instead of Block in + UnderReplicatedBlocks and PendingReplicationBlocks (Jing Zhao) + + HDFS-7369. Erasure coding: distribute recovery work for striped blocks to + DataNode (Zhe Zhang) + + HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks + (GAO Rui via jing9) + + HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage + ( Hui Zheng via jing9 ) + + HDFS-7616. Add a test for BlockGroup support in FSImage. (Takuya Fukudome via szetszwo) + HDFS-7907. Erasure Coding: track invalid, corrupt, and under-recovery striped + blocks in NameNode (Jing Zhao) + + HDFS-8005. Erasure Coding: simplify striped block recovery work computation + and add tests (Jing Zhao) \ No newline at end of file From 9edc73d1231e7eb773286eaa9a6302931faed9ca Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 31 Mar 2015 10:46:04 -0700 Subject: [PATCH 037/212] HDFS-7617. Add unit tests for editlog transactions for EC. Contributed by Hui Zheng. --- .../server/namenode/TestFSEditLogLoader.java | 158 ++++++++++++++++++ 1 file changed, 158 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index bc55d1271d3ab..051b74b4678d8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -39,14 +39,19 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; 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.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.PathUtils; import org.apache.log4j.Level; @@ -414,4 +419,157 @@ public void testFSEditLogOpCodes() throws IOException { fromByte(code), FSEditLogOpCodes.fromByte(code)); } } + + @Test + public void testAddNewStripedBlock() throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/ec"; + String testFile = "testfile_001"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser1"; + String clientMachine = "testMachine1"; + long blkId = 1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + short blockNum = HdfsConstants.NUM_DATA_BLOCKS; + short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; + + //set the storage policy of the directory + fs.mkdir(new Path(testDir), new FsPermission("755")); + fs.setStoragePolicy(new Path(testDir), + HdfsConstants.EC_STORAGE_POLICY_NAME); + + // Create a file with striped block + Path p = new Path(testFilePath); + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + fns.leaveSafeMode(); + + // Add a striped block to the file + BlockInfoStriped stripedBlk = new BlockInfoStriped( + new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.getStripedBlocksFeature().addBlock(stripedBlk); + fns.getEditLog().logAddBlock(testFilePath, file); + file.toCompleteFile(System.currentTimeMillis()); + + //If the block by loaded is the same as above it means that + //we have successfully applied the edit log to the fsimage. + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + + INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory() + .getINode(testFilePath); + + assertTrue(inodeLoaded.isWithStripedBlocks()); + + BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks(); + assertEquals(1, blks.length); + assertEquals(blkId, blks[0].getBlockId()); + assertEquals(blkNumBytes, blks[0].getNumBytes()); + assertEquals(timestamp, blks[0].getGenerationStamp()); + assertEquals(blockNum, blks[0].getDataBlockNum()); + assertEquals(parityNum, blks[0].getParityBlockNum()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testUpdateStripedBlocks() throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/ec"; + String testFile = "testfile_002"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser2"; + String clientMachine = "testMachine2"; + long blkId = 1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + short blockNum = HdfsConstants.NUM_DATA_BLOCKS; + short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; + + //set the storage policy of the directory + fs.mkdir(new Path(testDir), new FsPermission("755")); + fs.setStoragePolicy(new Path(testDir), + HdfsConstants.EC_STORAGE_POLICY_NAME); + + //create a file with striped blocks + Path p = new Path(testFilePath); + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoStriped stripedBlk = new BlockInfoStriped( + new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.getStripedBlocksFeature().addBlock(stripedBlk); + fns.getEditLog().logAddBlock(testFilePath, file); + file.toCompleteFile(System.currentTimeMillis()); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + fns.leaveSafeMode(); + + //update the last block + long newBlkNumBytes = 1024*8; + long newTimestamp = 1426222918+3600; + file.toUnderConstruction(clientName, clientMachine); + file.getLastBlock().setNumBytes(newBlkNumBytes); + file.getLastBlock().setGenerationStamp(newTimestamp); + fns.getEditLog().logUpdateBlocks(testFilePath, file, true); + file.toCompleteFile(System.currentTimeMillis()); + + //After the namenode restarts if the block by loaded is the same as above + //(new block size and timestamp) it means that we have successfully + //applied the edit log to the fsimage. + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + + INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory() + .getINode(testFilePath); + + assertTrue(inodeLoaded.isWithStripedBlocks()); + + BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks(); + assertEquals(1, blks.length); + assertEquals(blkId, blks[0].getBlockId()); + assertEquals(newBlkNumBytes, blks[0].getNumBytes()); + assertEquals(newTimestamp, blks[0].getGenerationStamp()); + assertEquals(blockNum, blks[0].getDataBlockNum()); + assertEquals(parityNum, blks[0].getParityBlockNum()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } } From 98ea3ec0da978e1d9f0a5f4f11bfc471a39de842 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Thu, 2 Apr 2015 05:12:35 +0800 Subject: [PATCH 038/212] HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin --- hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 +++ .../main/java/org/apache/hadoop/io/erasurecode/ECSchema.java | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index b69e69adfa77b..01280db2b18ab 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -29,3 +29,6 @@ HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng ( Kai Zheng ) + + HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin + ( Xinwei Qin via Kai Zheng ) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index 8dc3f4516e0f6..27be00e673277 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -64,7 +64,7 @@ public ECSchema(String schemaName, Map options) { } } catch (NumberFormatException e) { throw new IllegalArgumentException("Option value " + - options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + options.get(NUM_DATA_UNITS_KEY) + " for " + NUM_DATA_UNITS_KEY + " is found. It should be an integer"); } @@ -74,7 +74,7 @@ public ECSchema(String schemaName, Map options) { } } catch (NumberFormatException e) { throw new IllegalArgumentException("Option value " + - options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + options.get(NUM_PARITY_UNITS_KEY) + " for " + NUM_PARITY_UNITS_KEY + " is found. It should be an integer"); } From 578019d6a27bd3128fae1bdf7caadd8767b91e8c Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 2 Apr 2015 11:25:58 -0700 Subject: [PATCH 039/212] HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for conflicts from HDFS-6945). Contributed by Zhe Zhang. --- .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 97d8379f5c354..6941295f09d39 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 @@ -3722,7 +3722,7 @@ public void removeBlockFromMap(Block block) { private void removeFromExcessReplicateMap(Block block) { for (DatanodeStorageInfo info : blocksMap.getStorages(block)) { String uuid = info.getDatanodeDescriptor().getDatanodeUuid(); - LightWeightLinkedSet excessReplicas = excessReplicateMap.get(uuid); + LightWeightLinkedSet excessReplicas = excessReplicateMap.get(uuid); if (excessReplicas != null) { if (excessReplicas.remove(block)) { excessBlocksCount.decrementAndGet(); From 1af8c148626effe1b41fc536019fd3349f485d59 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 2 Apr 2015 22:38:29 -0700 Subject: [PATCH 040/212] HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones. Contributed by Zhe Zhang --- .../org/apache/hadoop/hdfs/DFSClient.java | 15 ++ .../hadoop/hdfs/protocol/ClientProtocol.java | 8 + ...amenodeProtocolServerSideTranslatorPB.java | 14 ++ .../ClientNamenodeProtocolTranslatorPB.java | 16 ++ .../server/common/HdfsServerConstants.java | 2 + .../namenode/ErasureCodingZoneManager.java | 112 +++++++++++++ .../hdfs/server/namenode/FSDirRenameOp.java | 2 + .../server/namenode/FSDirWriteFileOp.java | 11 +- .../hdfs/server/namenode/FSDirectory.java | 23 +++ .../hdfs/server/namenode/FSNamesystem.java | 40 +++++ .../hdfs/server/namenode/INodeFile.java | 5 +- .../server/namenode/NameNodeRpcServer.java | 16 ++ .../main/proto/ClientNamenodeProtocol.proto | 9 ++ .../hadoop/hdfs/TestBlockStoragePolicy.java | 3 - .../hadoop/hdfs/TestErasureCodingZones.java | 151 ++++++++++++++++++ .../TestBlockInitialEncoding.java | 75 --------- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 6 +- .../hdfs/server/namenode/TestFSImage.java | 23 ++- .../namenode/TestRecoverStripedBlocks.java | 7 +- 20 files changed, 426 insertions(+), 114 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 60e5577028501..0c6383b4dcb18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -2994,6 +2994,21 @@ public RemoteIterator listEncryptionZones() return new EncryptionZoneIterator(namenode, traceSampler); } + public void createErasureCodingZone(String src) + throws IOException { + checkOpen(); + TraceScope scope = getPathTraceScope("createErasureCodingZone", src); + try { + namenode.createErasureCodingZone(src); + } catch (RemoteException re) { + throw re.unwrapRemoteException(AccessControlException.class, + SafeModeException.class, + UnresolvedPathException.class); + } finally { + scope.close(); + } + } + public void setXAttr(String src, String name, byte[] value, EnumSet flag) throws IOException { checkOpen(); 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 55faf1662c6db..a94caad972a82 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 @@ -1362,6 +1362,14 @@ public EncryptionZone getEZForPath(String src) public BatchedEntries listEncryptionZones( long prevId) throws IOException; + /** + * Create an erasure coding zone (currently with hardcoded schema) + * TODO: Configurable and pluggable schemas (HDFS-7337) + */ + @Idempotent + public void createErasureCodingZone(String src) + throws IOException; + /** * Set xattr of a file or directory. * The name must be prefixed with the namespace followed by ".". For example, 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 ae5745a760cf0..5be9bc6d8d7fc 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 @@ -192,6 +192,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto; @@ -1390,6 +1392,18 @@ public ListEncryptionZonesResponseProto listEncryptionZones( } } + @Override + public CreateErasureCodingZoneResponseProto createErasureCodingZone( + RpcController controller, CreateErasureCodingZoneRequestProto req) + throws ServiceException { + try { + server.createErasureCodingZone(req.getSrc()); + return CreateErasureCodingZoneResponseProto.newBuilder().build(); + } catch (IOException e) { + throw new ServiceException(e); + } + } + @Override public SetXAttrResponseProto setXAttr(RpcController controller, SetXAttrRequestProto req) throws ServiceException { 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 22da083c98882..43a0322c25c5b 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 @@ -160,6 +160,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; @@ -1406,6 +1408,20 @@ public BatchedEntries listEncryptionZones(long id) } } + @Override + public void createErasureCodingZone(String src) + throws IOException { + final CreateErasureCodingZoneRequestProto.Builder builder = + CreateErasureCodingZoneRequestProto.newBuilder(); + builder.setSrc(src); + CreateErasureCodingZoneRequestProto req = builder.build(); + try { + rpcProxy.createErasureCodingZone(null, req); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + @Override public void setXAttr(String src, XAttr xAttr, EnumSet flag) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java index 26a7ab34ddb68..2ebc92713aebc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java @@ -396,4 +396,6 @@ enum BlockUCState { "raw.hdfs.crypto.file.encryption.info"; String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER = "security.hdfs.unreadable.by.superuser"; + public static final String XATTR_ERASURECODING_ZONE = + "raw.hdfs.erasurecoding.zone"; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java new file mode 100644 index 0000000000000..d4ff7c56fe142 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.hdfs.XAttrHelper; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; + +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE; + +/** + * Manages the list of erasure coding zones in the filesystem. + *

+ * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory + * lock being held for many operations. The FSDirectory lock should not be + * taken if the manager lock is already held. + * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager} + */ +public class ErasureCodingZoneManager { + private final FSDirectory dir; + + /** + * Construct a new ErasureCodingZoneManager. + * + * @param dir Enclosing FSDirectory + */ + public ErasureCodingZoneManager(FSDirectory dir) { + this.dir = dir; + } + + boolean getECPolicy(INodesInPath iip) { + assert dir.hasReadLock(); + Preconditions.checkNotNull(iip); + List inodes = iip.getReadOnlyINodes(); + for (int i = inodes.size() - 1; i >= 0; i--) { + final INode inode = inodes.get(i); + if (inode == null) { + continue; + } + final List xAttrs = inode.getXAttrFeature() == null ? + new ArrayList(0) + : inode.getXAttrFeature().getXAttrs(); + for (XAttr xAttr : xAttrs) { + if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { + return true; + } + } + } + return false; + } + + XAttr createErasureCodingZone(String src) + throws IOException { + assert dir.hasWriteLock(); + final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false); + if (dir.isNonEmptyDirectory(srcIIP)) { + throw new IOException( + "Attempt to create an erasure coding zone for a " + + "non-empty directory."); + } + if (srcIIP != null && + srcIIP.getLastINode() != null && + !srcIIP.getLastINode().isDirectory()) { + throw new IOException("Attempt to create an erasure coding zone " + + "for a file."); + } + if (getECPolicy(srcIIP)) { + throw new IOException("Directory " + src + " is already in an " + + "erasure coding zone."); + } + final XAttr ecXAttr = XAttrHelper + .buildXAttr(XATTR_ERASURECODING_ZONE, null); + final List xattrs = Lists.newArrayListWithCapacity(1); + xattrs.add(ecXAttr); + FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs, + EnumSet.of(XAttrSetFlag.CREATE)); + return ecXAttr; + } + + void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src) + throws IOException { + assert dir.hasReadLock(); + if (getECPolicy(srcIIP) + != getECPolicy(dstIIP)) { + throw new IOException( + src + " can't be moved because the source and destination have " + + "different erasure coding policies."); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java index b69bb4265b3f6..127474cc295e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java @@ -185,6 +185,7 @@ static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst, } fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src); + fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src); // Ensure dst has quota to accommodate rename verifyFsLimitsForRename(fsd, srcIIP, dstIIP); verifyQuotaForRename(fsd, srcIIP, dstIIP); @@ -357,6 +358,7 @@ static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst, BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite(); fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src); + fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src); final INode dstInode = dstIIP.getLastINode(); List snapshottableDirs = new ArrayList<>(); if (dstInode != null) { // Destination exists diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 325d9d34e8115..fa17d9df9b024 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -482,9 +483,6 @@ static INodeFile addFileForEditLog( try { INodesInPath iip = fsd.addINode(existing, newNode); if (iip != null) { - if (newNode.isStriped()) { - newNode.addStripedBlocksFeature(); - } if (aclEntries != null) { AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID); } @@ -560,9 +558,6 @@ private static INodesInPath addFile( fsd.writeLock(); try { newiip = fsd.addINode(existing, newNode); - if (newiip != null && newNode.isStriped()) { - newNode.addStripedBlocksFeature(); - } } finally { fsd.writeUnlock(); } @@ -610,7 +605,7 @@ private static FileState analyzeFileState( } } final INodeFile file = fsn.checkLease(src, clientName, inode, fileId); - BlockInfoContiguous lastBlockInFile = file.getLastBlock(); + BlockInfo lastBlockInFile = file.getLastBlock(); if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) { // The block that the client claims is the current last block // doesn't match up with what we think is the last block. There are @@ -638,7 +633,7 @@ private static FileState analyzeFileState( // changed the namesystem state yet. // We run this analysis again in Part II where case 4 is impossible. - BlockInfoContiguous penultimateBlock = file.getPenultimateBlock(); + BlockInfo penultimateBlock = file.getPenultimateBlock(); if (previous == null && lastBlockInFile != null && lastBlockInFile.getNumBytes() >= file.getPreferredBlockSize() && 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 1be2ad090f421..af5793fa82bbe 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 @@ -205,6 +205,9 @@ public int getWriteHoldCount() { @VisibleForTesting public final EncryptionZoneManager ezManager; + @VisibleForTesting + public final ErasureCodingZoneManager ecZoneManager; + /** * Caches frequently used file names used in {@link INode} to reuse * byte[] objects and reduce heap usage. @@ -296,6 +299,7 @@ public int getWriteHoldCount() { namesystem = ns; this.editLog = ns.getEditLog(); ezManager = new EncryptionZoneManager(this, conf); + ecZoneManager = new ErasureCodingZoneManager(this); } FSNamesystem getFSNamesystem() { @@ -1221,6 +1225,25 @@ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId, } } + XAttr createErasureCodingZone(String src) + throws IOException { + writeLock(); + try { + return ecZoneManager.createErasureCodingZone(src); + } finally { + writeUnlock(); + } + } + + public boolean getECPolicy(INodesInPath iip) { + readLock(); + try { + return ecZoneManager.getECPolicy(iip); + } finally { + readUnlock(); + } + } + static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException { INode inode = iip.getLastINode(); if (inode == null) { 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 5fac43cc25b32..fa874d151d433 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 @@ -7510,6 +7510,46 @@ BatchedListEntries listEncryptionZones(long prevId) } } + /** + * Create an erasure coding zone on directory src. + * + * @param src the path of a directory which will be the root of the + * erasure coding zone. The directory must be empty. + * @throws AccessControlException if the caller is not the superuser. + * @throws UnresolvedLinkException if the path can't be resolved. + * @throws SafeModeException if the Namenode is in safe mode. + */ + void createErasureCodingZone(final String srcArg, + final boolean logRetryCache) + throws IOException, UnresolvedLinkException, + SafeModeException, AccessControlException { + String src = srcArg; + HdfsFileStatus resultingStat = null; + checkSuperuserPrivilege(); + checkOperation(OperationCategory.WRITE); + final byte[][] pathComponents = + FSDirectory.getPathComponentsForReservedPath(src); + FSPermissionChecker pc = getPermissionChecker(); + writeLock(); + try { + checkSuperuserPrivilege(); + checkOperation(OperationCategory.WRITE); + checkNameNodeSafeMode("Cannot create erasure coding zone on " + src); + src = dir.resolvePath(pc, src, pathComponents); + + final XAttr ecXAttr = dir.createErasureCodingZone(src); + List xAttrs = Lists.newArrayListWithCapacity(1); + xAttrs.add(ecXAttr); + getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); + final INodesInPath iip = dir.getINodesInPath4Write(src, false); + resultingStat = dir.getAuditFileInfo(iip); + } finally { + writeUnlock(); + } + getEditLog().logSync(); + logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat); + } + void setXAttr(String src, XAttr xAttr, EnumSet flag, boolean logRetryCache) throws IOException { 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 51e24db657e92..032578a22602d 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 @@ -420,7 +420,7 @@ public short getPreferredBlockReplication() { } max = maxInSnapshot > max ? maxInSnapshot : max; } - return isStriped()? + return isStriped() ? HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max; } @@ -1114,8 +1114,7 @@ boolean isBlockInLatestSnapshot(BlockInfoContiguous block) { */ @VisibleForTesting @Override - // TODO: move erasure coding policy to file XAttr public boolean isStriped() { - return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID; + return getStripedBlocksFeature() != null; } } 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 0d416a6eb28dc..6e0333cc76bc9 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 @@ -1820,6 +1820,22 @@ public BatchedEntries listEncryptionZones( return namesystem.listEncryptionZones(prevId); } + @Override // ClientProtocol + public void createErasureCodingZone(String src) + throws IOException { + checkNNStartup(); + final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); + if (cacheEntry != null && cacheEntry.isSuccess()) { + return; + } + boolean success = false; + try { + namesystem.createErasureCodingZone(src, cacheEntry != null); + } finally { + RetryCache.setState(cacheEntry, success); + } + } + @Override // ClientProtocol public void setXAttr(String src, XAttr xAttr, EnumSet flag) throws IOException { 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 b44c556bbaa68..183aff89a41a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -714,6 +714,13 @@ message GetEditsFromTxidResponseProto { required EventsListProto eventsList = 1; } +message CreateErasureCodingZoneRequestProto { + required string src = 1; +} + +message CreateErasureCodingZoneResponseProto { +} + service ClientNamenodeProtocol { rpc getBlockLocations(GetBlockLocationsRequestProto) returns(GetBlockLocationsResponseProto); @@ -856,6 +863,8 @@ service ClientNamenodeProtocol { returns(ListEncryptionZonesResponseProto); rpc getEZForPath(GetEZForPathRequestProto) returns(GetEZForPathResponseProto); + rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto) + returns(CreateErasureCodingZoneResponseProto); rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto) returns(GetCurrentEditLogTxidResponseProto); rpc getEditsFromTxid(GetEditsFromTxidRequestProto) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java index 13214f34484ae..ea69f976a6f16 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java @@ -119,9 +119,6 @@ public void testDefaultPolicies() { expectedPolicyStrings.put(COLD, "BlockStoragePolicy{COLD:" + COLD + ", storageTypes=[ARCHIVE], " + "creationFallbacks=[], replicationFallbacks=[]}"); - expectedPolicyStrings.put(EC, - "BlockStoragePolicy{EC:" + EC + ", storageTypes=[DISK], " + - "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}"); expectedPolicyStrings.put(WARM, "BlockStoragePolicy{WARM:" + WARM + ", storageTypes=[DISK, ARCHIVE], " + "creationFallbacks=[DISK, ARCHIVE], " + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java new file mode 100644 index 0000000000000..49f08eefdf6f0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -0,0 +1,151 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INode; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class TestErasureCodingZones { + private final int NUM_OF_DATANODES = 3; + private Configuration conf; + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private static final int BLOCK_SIZE = 1024; + private FSNamesystem namesystem; + + @Before + public void setupCluster() throws IOException { + conf = new HdfsConfiguration(); + conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + cluster = new MiniDFSCluster.Builder(conf). + numDataNodes(NUM_OF_DATANODES).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + namesystem = cluster.getNamesystem(); + } + + @After + public void shutdownCluster() throws IOException { + cluster.shutdown(); + } + + @Test + public void testCreateECZone() + throws IOException, InterruptedException { + final Path testDir = new Path("/ec"); + fs.mkdir(testDir, FsPermission.getDirDefault()); + + /* Normal creation of an erasure coding zone */ + fs.getClient().createErasureCodingZone(testDir.toString()); + + /* Verify files under the zone are striped */ + final Path ECFilePath = new Path(testDir, "foo"); + fs.create(ECFilePath); + INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString()); + assertTrue(inode.asFile().isStriped()); + + /* Verify that EC zone cannot be created on non-empty dir */ + final Path notEmpty = new Path("/nonEmpty"); + fs.mkdir(notEmpty, FsPermission.getDirDefault()); + fs.create(new Path(notEmpty, "foo")); + try { + fs.getClient().createErasureCodingZone(notEmpty.toString()); + fail("Erasure coding zone on non-empty dir"); + } catch (IOException e) { + assertExceptionContains("erasure coding zone for a non-empty directory", e); + } + + /* Verify that nested EC zones cannot be created */ + final Path zone1 = new Path("/zone1"); + final Path zone2 = new Path(zone1, "zone2"); + fs.mkdir(zone1, FsPermission.getDirDefault()); + fs.getClient().createErasureCodingZone(zone1.toString()); + fs.mkdir(zone2, FsPermission.getDirDefault()); + try { + fs.getClient().createErasureCodingZone(zone2.toString()); + fail("Nested erasure coding zones"); + } catch (IOException e) { + assertExceptionContains("already in an erasure coding zone", e); + } + + /* Verify that EC zone cannot be created on a file */ + final Path fPath = new Path("/file"); + fs.create(fPath); + try { + fs.getClient().createErasureCodingZone(fPath.toString()); + fail("Erasure coding zone on file"); + } catch (IOException e) { + assertExceptionContains("erasure coding zone for a file", e); + } + } + + @Test + public void testMoveValidity() throws IOException, InterruptedException { + final Path srcECDir = new Path("/srcEC"); + final Path dstECDir = new Path("/dstEC"); + fs.mkdir(srcECDir, FsPermission.getDirDefault()); + fs.mkdir(dstECDir, FsPermission.getDirDefault()); + fs.getClient().createErasureCodingZone(srcECDir.toString()); + fs.getClient().createErasureCodingZone(dstECDir.toString()); + final Path srcFile = new Path(srcECDir, "foo"); + fs.create(srcFile); + + /* Verify that a file can be moved between 2 EC zones */ + try { + fs.rename(srcFile, dstECDir); + } catch (IOException e) { + fail("A file should be able to move between 2 EC zones " + e); + } + + // Move the file back + fs.rename(new Path(dstECDir, "foo"), srcECDir); + + /* Verify that a file cannot be moved from a non-EC dir to an EC zone */ + final Path nonECDir = new Path("/nonEC"); + fs.mkdir(nonECDir, FsPermission.getDirDefault()); + try { + fs.rename(srcFile, nonECDir); + fail("A file shouldn't be able to move from a non-EC dir to an EC zone"); + } catch (IOException e) { + assertExceptionContains("can't be moved because the source and " + + "destination have different erasure coding policies", e); + } + + /* Verify that a file cannot be moved from an EC zone to a non-EC dir */ + final Path nonECFile = new Path(nonECDir, "nonECFile"); + fs.create(nonECFile); + try { + fs.rename(nonECFile, dstECDir); + } catch (IOException e) { + assertExceptionContains("can't be moved because the source and " + + "destination have different erasure coding policies", e); + } + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java deleted file mode 100644 index a84f67b232b0c..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * 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 org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.*; -import org.apache.hadoop.hdfs.client.HdfsAdmin; -import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; -import org.apache.hadoop.hdfs.server.namenode.INode; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; - -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_ID; -import static org.junit.Assert.assertEquals; - -public class TestBlockInitialEncoding { - private final int NUM_OF_DATANODES = 3; - private Configuration conf; - private MiniDFSCluster cluster; - private DistributedFileSystem fs; - private static final int BLOCK_SIZE = 1024; - private HdfsAdmin dfsAdmin; - private FSNamesystem namesystem; - - @Before - public void setupCluster() throws IOException { - conf = new HdfsConfiguration(); - conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); - cluster = new MiniDFSCluster.Builder(conf). - numDataNodes(NUM_OF_DATANODES).build(); - cluster.waitActive(); - fs = cluster.getFileSystem(); - dfsAdmin = new HdfsAdmin(cluster.getURI(), conf); - namesystem = cluster.getNamesystem(); - } - - @After - public void shutdownCluster() throws IOException { - cluster.shutdown(); - } - - @Test - public void testBlockInitialEncoding() - throws IOException, InterruptedException { - final Path testDir = new Path("/test"); - fs.mkdir(testDir, FsPermission.getDirDefault()); - dfsAdmin.setStoragePolicy(testDir, EC_STORAGE_POLICY_NAME); - final Path ECFilePath = new Path("/test/foo.ec"); - DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, (short) 3, 0); - INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString()); - assertEquals(EC_STORAGE_POLICY_ID, inode.getStoragePolicyID()); - } - -} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 215a4e4b8efb1..c3c823936c887 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -68,7 +68,7 @@ public void setup() throws IOException { .numDataNodes(GROUP_SIZE).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); - dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME); + dfs.getClient().createErasureCodingZone("/"); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 051b74b4678d8..7dab7eb1280d5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -445,8 +445,7 @@ public void testAddNewStripedBlock() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.setStoragePolicy(new Path(testDir), - HdfsConstants.EC_STORAGE_POLICY_NAME); + fs.getClient().getNamenode().createErasureCodingZone(testDir); // Create a file with striped block Path p = new Path(testFilePath); @@ -518,8 +517,7 @@ public void testUpdateStripedBlocks() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.setStoragePolicy(new Path(testDir), - HdfsConstants.EC_STORAGE_POLICY_NAME); + fs.getClient().getNamenode().createErasureCodingZone(testDir); //create a file with striped blocks Path p = new Path(testFilePath); 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 16f247add78b2..7fdf1af05589a 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 @@ -33,18 +33,14 @@ import java.util.EnumSet; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.junit.Assert; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -59,7 +55,6 @@ 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.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease; import org.apache.hadoop.hdfs.util.MD5FileUtils; @@ -137,9 +132,10 @@ private void testPersistHelper(Configuration conf) throws IOException { } } - private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf, - boolean isUC) throws IOException{ + private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf, + boolean isUC) throws IOException{ // contruct a INode with StripedBlock for saving and loading + fsn.createErasureCodingZone("/", false); long id = 123456789; byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes(); PermissionStatus permissionStatus = new PermissionStatus("testuser_a", @@ -149,9 +145,8 @@ private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf, BlockInfoContiguous[] blks = new BlockInfoContiguous[0]; short replication = 3; long preferredBlockSize = 128*1024*1024; - byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID; INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime, - blks, replication, preferredBlockSize, storagePolicyID); + blks, replication, preferredBlockSize); ByteArrayOutputStream bs = new ByteArrayOutputStream(); file.addStripedBlocksFeature(); @@ -237,13 +232,13 @@ private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf, * FSImageSerialization and loaded by FSImageFormat#Loader. */ @Test - public void testSaveAndLoadInodeFile() throws IOException{ + public void testSaveAndLoadStripedINodeFile() throws IOException{ Configuration conf = new Configuration(); MiniDFSCluster cluster = null; try { cluster = new MiniDFSCluster.Builder(conf).build(); cluster.waitActive(); - testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false); + testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, false); } finally { if (cluster != null) { cluster.shutdown(); @@ -256,14 +251,14 @@ public void testSaveAndLoadInodeFile() throws IOException{ * saved and loaded by FSImageSerialization */ @Test - public void testSaveAndLoadInodeFileUC() throws IOException{ + public void testSaveAndLoadStripedINodeFileUC() throws IOException{ // construct a INode with StripedBlock for saving and loading Configuration conf = new Configuration(); MiniDFSCluster cluster = null; try { cluster = new MiniDFSCluster.Builder(conf).build(); cluster.waitActive(); - testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true); + testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true); } finally { if (cluster != null) { cluster.shutdown(); @@ -402,7 +397,7 @@ public void testSupportBlockGroup() throws IOException { .build(); cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); - fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME); + fs.getClient().getNamenode().createErasureCodingZone("/"); Path file = new Path("/striped"); FSDataOutputStream out = fs.create(file); byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index b9fd4fee16e72..d965ae721344c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.client.HdfsAdmin; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; @@ -37,23 +36,19 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.io.IOUtils; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.io.IOException; -import java.util.Iterator; import java.util.List; import java.util.UUID; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -93,7 +88,7 @@ public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, int numBlocks) throws Exception { DistributedFileSystem dfs = cluster.getFileSystem(); dfs.mkdirs(dir); - dfs.setStoragePolicy(dir, EC_STORAGE_POLICY_NAME); + dfs.getClient().getNamenode().createErasureCodingZone(dir.toString()); FSDataOutputStream out = null; try { From e54a74b566f89a424a2f4735a35553ece3bd35d9 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 3 Apr 2015 15:22:50 -0700 Subject: [PATCH 041/212] HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang. --- .../coder/AbstractErasureDecoder.java | 7 ++-- .../coder/AbstractErasureEncoder.java | 7 ++-- .../io/erasurecode/coder/ErasureCoder.java | 12 ++++++ .../io/erasurecode/coder/ErasureDecoder.java | 41 ------------------- .../io/erasurecode/coder/ErasureEncoder.java | 39 ------------------ .../coder/TestErasureCoderBase.java | 20 ++++----- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 14 ++++++- 7 files changed, 41 insertions(+), 99 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java index 54a6d1ed7a701..cd31294f80781 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java @@ -23,13 +23,12 @@ /** * An abstract erasure decoder that's to be inherited by new decoders. * - * It implements the {@link ErasureDecoder} interface. + * It implements the {@link ErasureCoder} interface. */ -public abstract class AbstractErasureDecoder extends AbstractErasureCoder - implements ErasureDecoder { +public abstract class AbstractErasureDecoder extends AbstractErasureCoder { @Override - public ErasureCodingStep decode(ECBlockGroup blockGroup) { + public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) { // We may have more than this when considering complicate cases. HADOOP-11550 return prepareDecodingStep(blockGroup); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java index 09b31e5bd24df..a836b75e7a58d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java @@ -23,13 +23,12 @@ /** * An abstract erasure encoder that's to be inherited by new encoders. * - * It implements the {@link ErasureEncoder} interface. + * It implements the {@link ErasureCoder} interface. */ -public abstract class AbstractErasureEncoder extends AbstractErasureCoder - implements ErasureEncoder { +public abstract class AbstractErasureEncoder extends AbstractErasureCoder { @Override - public ErasureCodingStep encode(ECBlockGroup blockGroup) { + public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) { // We may have more than this when considering complicate cases. HADOOP-11550 return prepareEncodingStep(blockGroup); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java index c5922f37b15f1..fb9015664b562 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java @@ -18,6 +18,7 @@ package org.apache.hadoop.io.erasurecode.coder; import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; /** * An erasure coder to perform encoding or decoding given a group. Generally it @@ -63,6 +64,17 @@ public interface ErasureCoder extends Configurable { */ public int getChunkSize(); + /** + * Calculate the encoding or decoding steps given a block blockGroup. + * + * Note, currently only one coding step is supported. Will support complex + * cases of multiple coding steps. + * + * @param blockGroup the erasure coding block group containing all necessary + * information for codec calculation + */ + public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup); + /** * Tell if native or off-heap buffer is preferred or not. It's for callers to * decide how to allocate coding chunk buffers, either on heap or off heap. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java deleted file mode 100644 index dfd9e545b502c..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * 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.io.erasurecode.coder; - - -import org.apache.hadoop.io.erasurecode.ECBlockGroup; - -/** - * Erasure decoder interface to perform decoding given a block group. - * - * It extends {@link ErasureCoder}. - */ -public interface ErasureDecoder extends ErasureCoder { - - /** - * Perform the decoding given a blockGroup. By default it will try the best to - * attempt to recover all the missing blocks according to the codec logic. - * - * Note, currently only one coding step is supported. Will support complex - * cases of multiple coding steps. - * - * @param blockGroup - */ - public ErasureCodingStep decode(ECBlockGroup blockGroup); - -} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java deleted file mode 100644 index e837d229dcab0..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * 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.io.erasurecode.coder; - -import org.apache.hadoop.io.erasurecode.ECBlockGroup; - -/** - * Erasure encoder interface to perform encoding given a block group. - * - * It extends {@link ErasureCoder}. - */ -public interface ErasureEncoder extends ErasureCoder { - - /** - * Calculate the encoding steps given a block blockGroup. - * - * Note, currently only one coding step is supported. Will support complex - * cases of multiple coding steps. - * - * @param blockGroup - */ - public ErasureCodingStep encode(ECBlockGroup blockGroup); - -} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index b963a594eb112..05a62a79f49cb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -26,8 +26,8 @@ * Erasure coder test base with utilities. */ public abstract class TestErasureCoderBase extends TestCoderBase { - protected Class encoderClass; - protected Class decoderClass; + protected Class encoderClass; + protected Class decoderClass; protected int numChunksInBlock = 16; @@ -55,7 +55,7 @@ public TestBlock(ECChunk[] chunks) { protected void testCoding(boolean usingDirectBuffer) { this.usingDirectBuffer = usingDirectBuffer; - ErasureEncoder encoder = createEncoder(); + ErasureCoder encoder = createEncoder(); // Generate data and encode ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); @@ -68,7 +68,7 @@ protected void testCoding(boolean usingDirectBuffer) { ErasureCodingStep codingStep; try { - codingStep = encoder.encode(blockGroup); + codingStep = encoder.calculateCoding(blockGroup); performCodingStep(codingStep); } finally { encoder.release(); @@ -78,9 +78,9 @@ protected void testCoding(boolean usingDirectBuffer) { //Decode blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks()); - ErasureDecoder decoder = createDecoder(); + ErasureCoder decoder = createDecoder(); try { - codingStep = decoder.decode(blockGroup); + codingStep = decoder.calculateCoding(blockGroup); performCodingStep(codingStep); } finally { decoder.release(); @@ -138,8 +138,8 @@ protected void compareAndVerify(ECBlock[] erasedBlocks, * Create erasure encoder for test. * @return */ - private ErasureEncoder createEncoder() { - ErasureEncoder encoder; + private ErasureCoder createEncoder() { + ErasureCoder encoder; try { encoder = encoderClass.newInstance(); } catch (Exception e) { @@ -155,8 +155,8 @@ private ErasureEncoder createEncoder() { * Create the erasure decoder for the test. * @return */ - private ErasureDecoder createDecoder() { - ErasureDecoder decoder; + private ErasureCoder createDecoder() { + ErasureCoder decoder; try { decoder = decoderClass.newInstance(); } catch (Exception e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index a686315142570..4e60a7c46d010 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -43,4 +43,16 @@ blocks in NameNode (Jing Zhao) HDFS-8005. Erasure Coding: simplify striped block recovery work computation - and add tests (Jing Zhao) \ No newline at end of file + and add tests (Jing Zhao) + + HDFS-7617. Add unit tests for editlog transactions for EC + (Hui Zheng via Zhe Zhang) + + HADOOP-11782. Correct two thrown messages in ECSchema class + (Xinwei Qin via Kai Zheng) + + HDFS-7839. Erasure coding: implement facilities in NameNode to create and + manage EC zones (Zhe Zhang) + + HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang) + From c243319eab89928bc06e0357a50f9422cbe54f7d Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 6 Apr 2015 10:37:23 -0700 Subject: [PATCH 042/212] HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8035). Contributed by Zhe Zhang --- .../hdfs/server/blockmanagement/BlockManager.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) 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 6941295f09d39..93134bde0f2c0 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 @@ -3584,13 +3584,12 @@ public boolean checkBlocksProperlyReplicated( String src, BlockInfo[] blocks) { for (BlockInfo b: blocks) { if (!b.isComplete()) { - final BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction)b; final int numNodes = b.numNodes(); - LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " - + uc.getBlockUCState() + ", replication# = " + numNodes - + (numNodes < minReplication ? " < ": " >= ") - + " minimum = " + minReplication + ") in file " + src); + final int min = getMinStorageNum(b); + final BlockUCState state = b.getBlockUCState(); + LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state + + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ") + + " minimum = " + min + ") in file " + src); return false; } } From 146ce7a9784e52432b76164009336a4b2cf2860e Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 6 Apr 2015 12:52:44 -0700 Subject: [PATCH 043/212] HDFS-7969. Erasure coding: NameNode support for lease recovery of striped block groups. Contributed by Zhe Zhang. --- .../BlockInfoContiguousUnderConstruction.java | 33 ++++---- .../BlockInfoStripedUnderConstruction.java | 80 +++++++++++++++---- .../BlockInfoUnderConstruction.java | 57 +++++++++++++ .../blockmanagement/DatanodeDescriptor.java | 12 +-- .../blockmanagement/DatanodeManager.java | 10 +-- .../hdfs/server/namenode/FSNamesystem.java | 24 +++--- .../TestBlockInfoUnderConstruction.java | 2 +- 7 files changed, 163 insertions(+), 55 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java index 7a052fd175fa1..9ba2978ac7be9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java @@ -31,7 +31,8 @@ * Represents a block that is currently being constructed.
* This is usually the last block of a file opened for write or append. */ -public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous { +public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous + implements BlockInfoUnderConstruction{ /** Block state. See {@link BlockUCState} */ private BlockUCState blockUCState; @@ -94,7 +95,7 @@ assert getBlockUCState() != BlockUCState.COMPLETE : return new BlockInfoContiguous(this); } - /** Set expected locations */ + @Override public void setExpectedLocations(DatanodeStorageInfo[] targets) { int numLocations = targets == null ? 0 : targets.length; this.replicas = new ArrayList<>(numLocations); @@ -104,10 +105,7 @@ public void setExpectedLocations(DatanodeStorageInfo[] targets) { } } - /** - * Create array of expected replica locations - * (as has been assigned by chooseTargets()). - */ + @Override public DatanodeStorageInfo[] getExpectedStorageLocations() { int numLocations = replicas == null ? 0 : replicas.size(); DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; @@ -117,7 +115,7 @@ public DatanodeStorageInfo[] getExpectedStorageLocations() { return storages; } - /** Get the number of expected locations */ + @Override public int getNumExpectedLocations() { return replicas == null ? 0 : replicas.size(); } @@ -135,25 +133,26 @@ void setBlockUCState(BlockUCState s) { blockUCState = s; } - /** Get block recovery ID */ + @Override public long getBlockRecoveryId() { return blockRecoveryId; } - /** Get recover block */ + @Override public Block getTruncateBlock() { return truncateBlock; } + @Override + public Block toBlock(){ + return this; + } + public void setTruncateBlock(Block recoveryBlock) { this.truncateBlock = recoveryBlock; } - /** - * Process the recorded replicas. When about to commit or finish the - * pipeline recovery sort out bad replicas. - * @param genStamp The final generation stamp for the block. - */ + @Override public void setGenerationStampAndVerifyReplicas(long genStamp) { // Set the generation stamp for the block. setGenerationStamp(genStamp); @@ -187,11 +186,7 @@ void commitBlock(Block block) throws IOException { setGenerationStampAndVerifyReplicas(block.getGenerationStamp()); } - /** - * Initialize lease recovery for this block. - * Find the first alive data-node starting from the previous primary and - * make it primary. - */ + @Override public void initializeBlockRecovery(long recoveryId) { setBlockUCState(BlockUCState.UNDER_RECOVERY); blockRecoveryId = recoveryId; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index b1857bbb88a4e..cfaf3a0f8caf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -31,7 +31,8 @@ * Represents a striped block that is currently being constructed. * This is usually the last block of a file opened for write or append. */ -public class BlockInfoStripedUnderConstruction extends BlockInfoStriped { +public class BlockInfoStripedUnderConstruction extends BlockInfoStriped + implements BlockInfoUnderConstruction{ private BlockUCState blockUCState; /** @@ -39,6 +40,12 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped { */ private ReplicaUnderConstruction[] replicas; + /** + * Index of the primary data node doing the recovery. Useful for log + * messages. + */ + private int primaryNodeIndex = -1; + /** * The new generation stamp, which this block will have * after the recovery succeeds. Also used as a recovery id to identify @@ -82,6 +89,7 @@ assert getBlockUCState() != COMPLETE : } /** Set expected locations */ + @Override public void setExpectedLocations(DatanodeStorageInfo[] targets) { int numLocations = targets == null ? 0 : targets.length; this.replicas = new ReplicaUnderConstruction[numLocations]; @@ -98,6 +106,7 @@ public void setExpectedLocations(DatanodeStorageInfo[] targets) { * Create array of expected replica locations * (as has been assigned by chooseTargets()). */ + @Override public DatanodeStorageInfo[] getExpectedStorageLocations() { int numLocations = getNumExpectedLocations(); DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; @@ -117,7 +126,7 @@ public int[] getBlockIndices() { return indices; } - /** Get the number of expected locations */ + @Override public int getNumExpectedLocations() { return replicas == null ? 0 : replicas.length; } @@ -135,16 +144,22 @@ void setBlockUCState(BlockUCState s) { blockUCState = s; } - /** Get block recovery ID */ + @Override public long getBlockRecoveryId() { return blockRecoveryId; } - /** - * Process the recorded replicas. When about to commit or finish the - * pipeline recovery sort out bad replicas. - * @param genStamp The final generation stamp for the block. - */ + @Override + public Block getTruncateBlock() { + return null; + } + + @Override + public Block toBlock(){ + return this; + } + + @Override public void setGenerationStampAndVerifyReplicas(long genStamp) { // Set the generation stamp for the block. setGenerationStamp(genStamp); @@ -178,18 +193,53 @@ void commitBlock(Block block) throws IOException { setGenerationStampAndVerifyReplicas(block.getGenerationStamp()); } - /** - * Initialize lease recovery for this striped block. - */ + @Override public void initializeBlockRecovery(long recoveryId) { setBlockUCState(BlockUCState.UNDER_RECOVERY); blockRecoveryId = recoveryId; if (replicas == null || replicas.length == 0) { NameNode.blockStateChangeLog.warn("BLOCK*" + - " BlockInfoUnderConstruction.initLeaseRecovery:" + + " BlockInfoStripedUnderConstruction.initLeaseRecovery:" + " No blocks found, lease removed."); } - // TODO we need to implement different recovery logic here + boolean allLiveReplicasTriedAsPrimary = true; + for (ReplicaUnderConstruction replica : replicas) { + // Check if all replicas have been tried or not. + if (replica.isAlive()) { + allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary && + replica.getChosenAsPrimary()); + } + } + if (allLiveReplicasTriedAsPrimary) { + // Just set all the replicas to be chosen whether they are alive or not. + for (ReplicaUnderConstruction replica : replicas) { + replica.setChosenAsPrimary(false); + } + } + long mostRecentLastUpdate = 0; + ReplicaUnderConstruction primary = null; + primaryNodeIndex = -1; + for(int i = 0; i < replicas.length; i++) { + // Skip alive replicas which have been chosen for recovery. + if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) { + continue; + } + final ReplicaUnderConstruction ruc = replicas[i]; + final long lastUpdate = ruc.getExpectedStorageLocation() + .getDatanodeDescriptor().getLastUpdateMonotonic(); + if (lastUpdate > mostRecentLastUpdate) { + primaryNodeIndex = i; + primary = ruc; + mostRecentLastUpdate = lastUpdate; + } + } + if (primary != null) { + primary.getExpectedStorageLocation().getDatanodeDescriptor() + .addBlockToBeRecovered(this); + primary.setChosenAsPrimary(true); + NameNode.blockStateChangeLog.info( + "BLOCK* {} recovery started, primary={}", this, primary); + } } void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock, @@ -238,7 +288,9 @@ public void appendStringTo(StringBuilder sb) { } private void appendUCParts(StringBuilder sb) { - sb.append("{UCState=").append(blockUCState).append(", replicas=["); + sb.append("{UCState=").append(blockUCState). + append(", primaryNodeIndex=").append(primaryNodeIndex). + append(", replicas=["); if (replicas != null) { int i = 0; for (ReplicaUnderConstruction r : replicas) { 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 new file mode 100644 index 0000000000000..bfdd3864a5ecb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java @@ -0,0 +1,57 @@ +/** + * 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 org.apache.hadoop.hdfs.protocol.Block; + +public interface BlockInfoUnderConstruction { + /** + * Create array of expected replica locations + * (as has been assigned by chooseTargets()). + */ + public DatanodeStorageInfo[] getExpectedStorageLocations(); + + /** Get recover block */ + public Block getTruncateBlock(); + + /** Convert to a Block object */ + public Block toBlock(); + + /** Get block recovery ID */ + public long getBlockRecoveryId(); + + /** Get the number of expected locations */ + public int getNumExpectedLocations(); + + /** Set expected locations */ + public void setExpectedLocations(DatanodeStorageInfo[] targets); + + /** + * Process the recorded replicas. When about to commit or finish the + * pipeline recovery sort out bad replicas. + * @param genStamp The final generation stamp for the block. + */ + public void setGenerationStampAndVerifyReplicas(long genStamp); + + /** + * Initialize lease recovery for this block. + * Find the first alive data-node starting from the previous primary and + * make it primary. + */ + public void initializeBlockRecovery(long recoveryId); +} 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 15427f7590632..7ec71a27e090b 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 @@ -253,8 +253,8 @@ public CachedBlocksList getPendingUncached() { private final BlockQueue erasurecodeBlocks = new BlockQueue<>(); /** A queue of blocks to be recovered by this datanode */ - private final BlockQueue - recoverBlocks = new BlockQueue<>(); + private final BlockQueue recoverBlocks = + new BlockQueue<>(); /** A set of blocks to be invalidated by this datanode */ private final LightWeightHashSet invalidateBlocks = new LightWeightHashSet<>(); @@ -649,7 +649,7 @@ void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources, /** * Store block recovery work. */ - void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) { + void addBlockToBeRecovered(BlockInfoUnderConstruction block) { if(recoverBlocks.contains(block)) { // this prevents adding the same block twice to the recovery queue BlockManager.LOG.info(block + " is already in the recovery queue"); @@ -703,11 +703,11 @@ public List getErasureCodeCommand(int maxTransfers) { return erasurecodeBlocks.poll(maxTransfers); } - public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { - List blocks = recoverBlocks.poll(maxTransfers); + public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { + List blocks = recoverBlocks.poll(maxTransfers); if(blocks == null) return null; - return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]); + return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]); } /** 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 c63e657df6059..8a78a0be99136 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 @@ -1379,12 +1379,12 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, } //check lease recovery - BlockInfoContiguousUnderConstruction[] blocks = nodeinfo + BlockInfoUnderConstruction[] blocks = nodeinfo .getLeaseRecoveryCommand(Integer.MAX_VALUE); if (blocks != null) { BlockRecoveryCommand brCommand = new BlockRecoveryCommand( blocks.length); - for (BlockInfoContiguousUnderConstruction b : blocks) { + for (BlockInfoUnderConstruction b : blocks) { final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations(); // Skip stale nodes during recovery - not heart beated for some time (30s by default). final List recoveryLocations = @@ -1398,10 +1398,10 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, // to old block. boolean truncateRecovery = b.getTruncateBlock() != null; boolean copyOnTruncateRecovery = truncateRecovery && - b.getTruncateBlock().getBlockId() != b.getBlockId(); + b.getTruncateBlock().getBlockId() != b.toBlock().getBlockId(); ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ? new ExtendedBlock(blockPoolId, b.getTruncateBlock()) : - new ExtendedBlock(blockPoolId, b); + new ExtendedBlock(blockPoolId, b.toBlock()); // If we only get 1 replica after eliminating stale nodes, then choose all // replicas for recovery and let the primary data node handle failures. DatanodeInfo[] recoveryInfos; @@ -1418,7 +1418,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages); } if(truncateRecovery) { - Block recoveryBlock = (copyOnTruncateRecovery) ? b : + Block recoveryBlock = (copyOnTruncateRecovery) ? b.toBlock() : b.getTruncateBlock(); brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos, recoveryBlock)); 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 fa874d151d433..d19a1db817516 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 @@ -208,6 +208,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +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; @@ -3567,18 +3568,19 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, throw new AlreadyBeingCreatedException(message); case UNDER_CONSTRUCTION: case UNDER_RECOVERY: - // TODO support Striped block's recovery - final BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction)lastBlock; + // TODO support truncate of striped blocks + final BlockInfoUnderConstruction uc = + (BlockInfoUnderConstruction)lastBlock; // determine if last block was intended to be truncated Block recoveryBlock = uc.getTruncateBlock(); boolean truncateRecovery = recoveryBlock != null; boolean copyOnTruncate = truncateRecovery && - recoveryBlock.getBlockId() != uc.getBlockId(); + recoveryBlock.getBlockId() != uc.toBlock().getBlockId(); assert !copyOnTruncate || - recoveryBlock.getBlockId() < uc.getBlockId() && - recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() && - recoveryBlock.getNumBytes() > uc.getNumBytes() : + recoveryBlock.getBlockId() < uc.toBlock().getBlockId() && + recoveryBlock.getGenerationStamp() < uc.toBlock(). + getGenerationStamp() && + recoveryBlock.getNumBytes() > uc.toBlock().getNumBytes() : "wrong recoveryBlock"; // setup the last block locations from the blockManager if not known @@ -3586,7 +3588,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, uc.setExpectedLocations(blockManager.getStorages(lastBlock)); } - if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) { + if (uc.getNumExpectedLocations() == 0 && + uc.toBlock().getNumBytes() == 0) { // There is no datanode reported to this block. // may be client have crashed before writing data to pipeline. // This blocks doesn't need any recovery. @@ -3599,10 +3602,11 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, return true; } // start recovery of the last block for this file - long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc)); + long blockRecoveryId = + nextGenerationStamp(blockIdManager.isLegacyBlock(uc.toBlock())); lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile); if(copyOnTruncate) { - uc.setGenerationStamp(blockRecoveryId); + uc.toBlock().setGenerationStamp(blockRecoveryId); } else if(truncateRecovery) { recoveryBlock.setGenerationStamp(blockRecoveryId); } 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 a7ba29399dcad..f5a9cc415822c 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 @@ -51,7 +51,7 @@ public void testInitializeBlockRecovery() throws Exception { DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000); blockInfo.initializeBlockRecovery(1); - BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1); + BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1); assertEquals(blockInfoRecovery[0], blockInfo); // Recovery attempt #2. From 17f7cdc04764524c091bb0e9eb43399f88ac0e6b Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Wed, 8 Apr 2015 01:26:40 +0800 Subject: [PATCH 044/212] HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng --- .../hadoop/io/erasurecode/coder/RSErasureDecoder.java | 8 ++++---- .../hadoop/io/erasurecode/coder/RSErasureEncoder.java | 4 ++-- .../{XorErasureDecoder.java => XORErasureDecoder.java} | 6 +++--- .../{XorErasureEncoder.java => XORErasureEncoder.java} | 6 +++--- .../rawcoder/{JRSRawDecoder.java => RSRawDecoder.java} | 2 +- .../rawcoder/{JRSRawEncoder.java => RSRawEncoder.java} | 2 +- ...ureCoderFactory.java => RSRawErasureCoderFactory.java} | 6 +++--- .../rawcoder/{XorRawDecoder.java => XORRawDecoder.java} | 2 +- .../rawcoder/{XorRawEncoder.java => XORRawEncoder.java} | 2 +- ...reCoderFactory.java => XORRawErasureCoderFactory.java} | 6 +++--- .../hadoop/io/erasurecode/coder/TestRSErasureCoder.java | 4 ++-- .../coder/{TestXorCoder.java => TestXORCoder.java} | 6 +++--- .../{TestJRSRawCoder.java => TestRSRawCoder.java} | 6 +++--- .../{TestXorRawCoder.java => TestXORRawCoder.java} | 8 +++----- 14 files changed, 33 insertions(+), 35 deletions(-) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/{XorErasureDecoder.java => XORErasureDecoder.java} (93%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/{XorErasureEncoder.java => XORErasureEncoder.java} (89%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/{JRSRawDecoder.java => RSRawDecoder.java} (97%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/{JRSRawEncoder.java => RSRawEncoder.java} (97%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/{JRSRawErasureCoderFactory.java => RSRawErasureCoderFactory.java} (88%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/{XorRawDecoder.java => XORRawDecoder.java} (97%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/{XorRawEncoder.java => XORRawEncoder.java} (96%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/{XorRawErasureCoderFactory.java => XORRawErasureCoderFactory.java} (89%) rename hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/{TestXorCoder.java => TestXORCoder.java} (89%) rename hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/{TestJRSRawCoder.java => TestRSRawCoder.java} (94%) rename hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/{TestXorRawCoder.java => TestXORRawCoder.java} (87%) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java index ba32f048aded0..e2c5051ab198c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -4,9 +4,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; -import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder; /** * Reed-Solomon erasure decoder that decodes a block group. @@ -56,7 +56,7 @@ private RawErasureDecoder checkCreateRSRawDecoder() { rsRawDecoder = createRawDecoder( CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY); if (rsRawDecoder == null) { - rsRawDecoder = new JRSRawDecoder(); + rsRawDecoder = new RSRawDecoder(); } rsRawDecoder.initialize(getNumDataUnits(), getNumParityUnits(), getChunkSize()); @@ -66,7 +66,7 @@ private RawErasureDecoder checkCreateRSRawDecoder() { private RawErasureDecoder checkCreateXorRawDecoder() { if (xorRawDecoder == null) { - xorRawDecoder = new XorRawDecoder(); + xorRawDecoder = new XORRawDecoder(); xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize()); } return xorRawDecoder; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java index 430749d3de161..a7d02b5ab6e1d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java @@ -3,7 +3,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; -import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; /** @@ -30,7 +30,7 @@ private RawErasureEncoder checkCreateRSRawEncoder() { rawEncoder = createRawEncoder( CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY); if (rawEncoder == null) { - rawEncoder = new JRSRawEncoder(); + rawEncoder = new RSRawEncoder(); } rawEncoder.initialize(getNumDataUnits(), getNumParityUnits(), getChunkSize()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java similarity index 93% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java index 33f53866ff0f8..6f4b4238986bb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java @@ -20,19 +20,19 @@ import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder; /** * Xor erasure decoder that decodes a block group. * * It implements {@link ErasureDecoder}. */ -public class XorErasureDecoder extends AbstractErasureDecoder { +public class XORErasureDecoder extends AbstractErasureDecoder { @Override protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { // May be configured - RawErasureDecoder rawDecoder = new XorRawDecoder(); + RawErasureDecoder rawDecoder = new XORRawDecoder(); rawDecoder.initialize(getNumDataUnits(), getNumParityUnits(), getChunkSize()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java similarity index 89% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java index f8d67c342c92e..90118573893e9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java @@ -20,19 +20,19 @@ import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; -import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder; /** * Xor erasure encoder that encodes a block group. * * It implements {@link ErasureEncoder}. */ -public class XorErasureEncoder extends AbstractErasureEncoder { +public class XORErasureEncoder extends AbstractErasureEncoder { @Override protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { // May be configured - RawErasureEncoder rawEncoder = new XorRawEncoder(); + RawErasureEncoder rawEncoder = new XORRawEncoder(); rawEncoder.initialize(getNumDataUnits(), getNumParityUnits(), getChunkSize()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java similarity index 97% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java index dbb689eec752c..24fa637426b7a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java @@ -26,7 +26,7 @@ * isn't available in some environment. Please always use native implementations * when possible. */ -public class JRSRawDecoder extends AbstractRawErasureDecoder { +public class RSRawDecoder extends AbstractRawErasureDecoder { // To describe and calculate the needed Vandermonde matrix private int[] errSignature; private int[] primitivePower; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java similarity index 97% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java index 6ea7551f26e23..7b501ceb453ca 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java @@ -26,7 +26,7 @@ * isn't available in some environment. Please always use native implementations * when possible. */ -public class JRSRawEncoder extends AbstractRawErasureEncoder { +public class RSRawEncoder extends AbstractRawErasureEncoder { private int[] generatingPolynomial; @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java similarity index 88% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java index d6b40aac17b9e..19a95af7ad4c8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java @@ -20,15 +20,15 @@ /** * A raw coder factory for raw Reed-Solomon coder in Java. */ -public class JRSRawErasureCoderFactory implements RawErasureCoderFactory { +public class RSRawErasureCoderFactory implements RawErasureCoderFactory { @Override public RawErasureEncoder createEncoder() { - return new JRSRawEncoder(); + return new RSRawEncoder(); } @Override public RawErasureDecoder createDecoder() { - return new JRSRawDecoder(); + return new RSRawDecoder(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java similarity index 97% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java index 98307a7b3c2f6..b6b163392e26f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java @@ -22,7 +22,7 @@ /** * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID. */ -public class XorRawDecoder extends AbstractRawErasureDecoder { +public class XORRawDecoder extends AbstractRawErasureDecoder { @Override protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java similarity index 96% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java index 99b20b92e7ad4..dbfab5d2f65e5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java @@ -22,7 +22,7 @@ /** * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID. */ -public class XorRawEncoder extends AbstractRawErasureEncoder { +public class XORRawEncoder extends AbstractRawErasureEncoder { @Override protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java similarity index 89% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java index 751d16f9e7967..67f45c161b5bd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java @@ -20,15 +20,15 @@ /** * A raw coder factory for raw XOR coder. */ -public class XorRawErasureCoderFactory implements RawErasureCoderFactory { +public class XORRawErasureCoderFactory implements RawErasureCoderFactory { @Override public RawErasureEncoder createEncoder() { - return new XorRawEncoder(); + return new XORRawEncoder(); } @Override public RawErasureDecoder createDecoder() { - return new XorRawDecoder(); + return new XORRawDecoder(); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java index 8a7561c1e2faa..3507dd2cb7b8f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java @@ -19,7 +19,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawErasureCoderFactory; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory; import org.junit.Before; import org.junit.Test; @@ -58,7 +58,7 @@ public void testCodingDirectBufferWithConf_10x4() { */ Configuration conf = new Configuration(); conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, - JRSRawErasureCoderFactory.class.getCanonicalName()); + RSRawErasureCoderFactory.class.getCanonicalName()); conf.setBoolean( CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false); prepare(conf, 10, 4, null); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java similarity index 89% rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java index d46fe061a5c5e..109e46ebb3804 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java @@ -23,12 +23,12 @@ /** * Test XOR encoding and decoding. */ -public class TestXorCoder extends TestErasureCoderBase { +public class TestXORCoder extends TestErasureCoderBase { @Before public void setup() { - this.encoderClass = XorErasureEncoder.class; - this.decoderClass = XorErasureDecoder.class; + this.encoderClass = XORErasureEncoder.class; + this.decoderClass = XORErasureDecoder.class; this.numDataUnits = 10; this.numParityUnits = 1; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java similarity index 94% rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 39e5deb452765..8bb5d0fc3183f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -27,7 +27,7 @@ /** * Test raw Reed-solomon encoding and decoding. */ -public class TestJRSRawCoder extends TestRawCoderBase { +public class TestRSRawCoder extends TestRawCoderBase { private static int symbolSize = 0; private static int symbolMax = 0; @@ -40,8 +40,8 @@ public class TestJRSRawCoder extends TestRawCoderBase { @Before public void setup() { - this.encoderClass = JRSRawEncoder.class; - this.decoderClass = JRSRawDecoder.class; + this.encoderClass = RSRawEncoder.class; + this.decoderClass = RSRawDecoder.class; } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java similarity index 87% rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java index ff4858616d4f3..e66e48d76a60b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java @@ -20,17 +20,15 @@ import org.junit.Before; import org.junit.Test; -import java.util.Random; - /** * Test XOR encoding and decoding. */ -public class TestXorRawCoder extends TestRawCoderBase { +public class TestXORRawCoder extends TestRawCoderBase { @Before public void setup() { - this.encoderClass = XorRawEncoder.class; - this.decoderClass = XorRawDecoder.class; + this.encoderClass = XORRawEncoder.class; + this.decoderClass = XORRawDecoder.class; this.numDataUnits = 10; this.numParityUnits = 1; From d43102d2c5960297b96eab7d82fb2e41e77021b4 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Wed, 8 Apr 2015 01:31:46 +0800 Subject: [PATCH 045/212] Updated CHANGES-HDFS-EC-7285.txt --- hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 01280db2b18ab..68d1d3284226f 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -32,3 +32,6 @@ HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin ( Xinwei Qin via Kai Zheng ) + + HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng + ( Kai Zheng ) From d25ca09cf629c64ad3cf3b67f3bc6e561353fdf2 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 7 Apr 2015 15:34:37 +0530 Subject: [PATCH 046/212] HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin Updated CHANGES-HDFS-EC-7285.txt --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 4e60a7c46d010..3874cb4728fba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -48,9 +48,6 @@ HDFS-7617. Add unit tests for editlog transactions for EC (Hui Zheng via Zhe Zhang) - HADOOP-11782. Correct two thrown messages in ECSchema class - (Xinwei Qin via Kai Zheng) - HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones (Zhe Zhang) From aac73c21c3e3f7552a3b32cd7238108b83a8fbe3 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 7 Apr 2015 15:35:18 +0530 Subject: [PATCH 047/212] HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang. Updated CHANGES-HDFS-EC-7285.txt --- hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 ++ hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +---- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 68d1d3284226f..77167282adf36 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -33,5 +33,7 @@ HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin ( Xinwei Qin via Kai Zheng ) + HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang) + HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng ( Kai Zheng ) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 3874cb4728fba..9927ccff50aa3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -49,7 +49,4 @@ (Hui Zheng via Zhe Zhang) HDFS-7839. Erasure coding: implement facilities in NameNode to create and - manage EC zones (Zhe Zhang) - - HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang) - + manage EC zones (Zhe Zhang) \ No newline at end of file From 99502cbbe28425fa4eaf42630218b775df1a54f7 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 7 Apr 2015 16:05:22 +0530 Subject: [PATCH 048/212] HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code ( Contributed by Kai Zheng) --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/io/erasurecode/ECBlockGroup.java | 18 ++++ .../codec/AbstractErasureCodec.java | 88 ++++++++++++++++++ .../io/erasurecode/codec/ErasureCodec.java | 56 ++++++++++++ .../io/erasurecode/codec/RSErasureCodec.java | 38 ++++++++ .../io/erasurecode/codec/XORErasureCodec.java | 45 ++++++++++ .../coder/AbstractErasureCoder.java | 7 ++ .../io/erasurecode/coder/ErasureCoder.java | 7 ++ .../io/erasurecode/grouper/BlockGrouper.java | 90 +++++++++++++++++++ 9 files changed, 352 insertions(+) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 77167282adf36..c72394e1b1c59 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -37,3 +37,6 @@ HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng ( Kai Zheng ) + + HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code + ( Kai Zheng via vinayakumarb ) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java index 2c851a50dfbb5..0a869075ac620 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java @@ -79,4 +79,22 @@ public boolean anyErasedParityBlock() { return false; } + /** + * Get erased blocks count + * @return + */ + public int getErasedCount() { + int erasedCount = 0; + + for (ECBlock dataBlock : dataBlocks) { + if (dataBlock.isErased()) erasedCount++; + } + + for (ECBlock parityBlock : parityBlocks) { + if (parityBlock.isErased()) erasedCount++; + } + + return erasedCount; + } + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java new file mode 100644 index 0000000000000..999378689384b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java @@ -0,0 +1,88 @@ +/** + * 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.io.erasurecode.codec; + +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.coder.*; +import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper; + +/** + * Abstract Erasure Codec that implements {@link ErasureCodec}. + */ +public abstract class AbstractErasureCodec extends Configured + implements ErasureCodec { + + private ECSchema schema; + + @Override + public void setSchema(ECSchema schema) { + this.schema = schema; + } + + public String getName() { + return schema.getCodecName(); + } + + protected ECSchema getSchema() { + return schema; + } + + @Override + public BlockGrouper createBlockGrouper() { + BlockGrouper blockGrouper = new BlockGrouper(); + blockGrouper.setSchema(getSchema()); + + return blockGrouper; + } + + @Override + public ErasureCoder createEncoder() { + ErasureCoder encoder = doCreateEncoder(); + prepareErasureCoder(encoder); + return encoder; + } + + /** + * Create a new encoder instance to be initialized afterwards. + * @return encoder + */ + protected abstract ErasureCoder doCreateEncoder(); + + @Override + public ErasureCoder createDecoder() { + ErasureCoder decoder = doCreateDecoder(); + prepareErasureCoder(decoder); + return decoder; + } + + /** + * Create a new decoder instance to be initialized afterwards. + * @return decoder + */ + protected abstract ErasureCoder doCreateDecoder(); + + private void prepareErasureCoder(ErasureCoder erasureCoder) { + if (getSchema() == null) { + throw new RuntimeException("No schema been set yet"); + } + + erasureCoder.setConf(getConf()); + erasureCoder.initialize(getSchema()); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java new file mode 100644 index 0000000000000..e639484ee2b17 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java @@ -0,0 +1,56 @@ +/** + * 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.io.erasurecode.codec; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.coder.ErasureCoder; +import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper; + +/** + * Erasure Codec API that's to cover the essential specific aspects of a code. + * Currently it cares only block grouper and erasure coder. In future we may + * add more aspects here to make the behaviors customizable. + */ +public interface ErasureCodec extends Configurable { + + /** + * Set EC schema to be used by this codec. + * @param schema + */ + public void setSchema(ECSchema schema); + + /** + * Create block grouper + * @return block grouper + */ + public BlockGrouper createBlockGrouper(); + + /** + * Create Erasure Encoder + * @return erasure encoder + */ + public ErasureCoder createEncoder(); + + /** + * Create Erasure Decoder + * @return erasure decoder + */ + public ErasureCoder createDecoder(); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java new file mode 100644 index 0000000000000..9e91b6019ce61 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java @@ -0,0 +1,38 @@ +/** + * 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.io.erasurecode.codec; + +import org.apache.hadoop.io.erasurecode.coder.ErasureCoder; +import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder; +import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder; + +/** + * A Reed-Solomon erasure codec. + */ +public class RSErasureCodec extends AbstractErasureCodec { + + @Override + protected ErasureCoder doCreateEncoder() { + return new RSErasureEncoder(); + } + + @Override + protected ErasureCoder doCreateDecoder() { + return new RSErasureDecoder(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java new file mode 100644 index 0000000000000..0f726d7cd0bf7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java @@ -0,0 +1,45 @@ +/** + * 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.io.erasurecode.codec; + +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.coder.ErasureCoder; +import org.apache.hadoop.io.erasurecode.coder.XORErasureDecoder; +import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder; + +/** + * A XOR erasure codec. + */ +public class XORErasureCodec extends AbstractErasureCodec { + + @Override + public void setSchema(ECSchema schema) { + super.setSchema(schema); + assert(schema.getNumParityUnits() == 1); + } + + @Override + protected ErasureCoder doCreateEncoder() { + return new XORErasureEncoder(); + } + + @Override + protected ErasureCoder doCreateDecoder() { + return new XORErasureDecoder(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index 0e4de8902a90a..e5bf11a2ac67e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -19,6 +19,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; @@ -104,6 +105,12 @@ public void initialize(int numDataUnits, int numParityUnits, this.chunkSize = chunkSize; } + @Override + public void initialize(ECSchema schema) { + initialize(schema.getNumDataUnits(), schema.getNumParityUnits(), + schema.getChunkSize()); + } + @Override public int getNumDataUnits() { return numDataUnits; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java index fb9015664b562..64a82eaf6ae8a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java @@ -19,6 +19,7 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; /** * An erasure coder to perform encoding or decoding given a group. Generally it @@ -44,6 +45,12 @@ public interface ErasureCoder extends Configurable { */ public void initialize(int numDataUnits, int numParityUnits, int chunkSize); + /** + * Initialize with an EC schema. + * @param schema + */ + public void initialize(ECSchema schema); + /** * The number of data input units for the coding. A unit can be a byte, * chunk or buffer or even a block. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java new file mode 100644 index 0000000000000..bdc162478892f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java @@ -0,0 +1,90 @@ +/** + * 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.io.erasurecode.grouper; + +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * As part of a codec, to handle how to form a block group for encoding + * and provide instructions on how to recover erased blocks from a block group + */ +public class BlockGrouper { + + private ECSchema schema; + + /** + * Set EC schema. + * @param schema + */ + public void setSchema(ECSchema schema) { + this.schema = schema; + } + + /** + * Get EC schema. + * @return + */ + protected ECSchema getSchema() { + return schema; + } + + /** + * Get required data blocks count in a BlockGroup. + * @return count of required data blocks + */ + public int getRequiredNumDataBlocks() { + return schema.getNumDataUnits(); + } + + /** + * Get required parity blocks count in a BlockGroup. + * @return count of required parity blocks + */ + public int getRequiredNumParityBlocks() { + return schema.getNumParityUnits(); + } + + /** + * Calculating and organizing BlockGroup, to be called by ECManager + * @param dataBlocks Data blocks to compute parity blocks against + * @param parityBlocks To be computed parity blocks + * @return + */ + public ECBlockGroup makeBlockGroup(ECBlock[] dataBlocks, + ECBlock[] parityBlocks) { + + ECBlockGroup blockGroup = new ECBlockGroup(dataBlocks, parityBlocks); + return blockGroup; + } + + /** + * Given a BlockGroup, tell if any of the missing blocks can be recovered, + * to be called by ECManager + * @param blockGroup a blockGroup that may contain erased blocks but not sure + * recoverable or not + * @return true if any erased block recoverable, false otherwise + */ + public boolean anyRecoverable(ECBlockGroup blockGroup) { + int erasedCount = blockGroup.getErasedCount(); + + return erasedCount > 0 && erasedCount <= getRequiredNumParityBlocks(); + } + +} From 91c741a2a171129638071306482c019d007972ab Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 7 Apr 2015 11:20:13 -0700 Subject: [PATCH 049/212] HDFS-7782. Erasure coding: pread from files in striped layout. Contributed by Zhe Zhang and Jing Zhao --- .../hadoop/hdfs/protocol/LocatedBlock.java | 4 + .../org/apache/hadoop/hdfs/DFSClient.java | 55 +++ .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 8 +- .../apache/hadoop/hdfs/DFSInputStream.java | 80 +++- .../hadoop/hdfs/DFSStripedInputStream.java | 367 ++++++++++++++++++ .../hdfs/protocol/LocatedStripedBlock.java | 5 + .../blockmanagement/BlockInfoStriped.java | 6 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 91 ++++- .../hadoop/hdfs/TestReadStripedFile.java | 304 +++++++++++++++ .../namenode/TestRecoverStripedBlocks.java | 88 +---- 10 files changed, 896 insertions(+), 112 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java index 4e8f2025be971..a9596bf67dadd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java @@ -203,4 +203,8 @@ public String toString() { + "; locs=" + Arrays.asList(locs) + "}"; } + + public boolean isStriped() { + return false; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 0c6383b4dcb18..a845fdfbf4395 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -238,6 +238,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, private static final DFSHedgedReadMetrics HEDGED_READ_METRIC = new DFSHedgedReadMetrics(); private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL; + private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL; private final Sampler traceSampler; private final int smallBufferSize; @@ -380,6 +381,19 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) { this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize()); } + numThreads = conf.getInt( + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE, + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE); + if (numThreads <= 0) { + LOG.warn("The value of " + + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE + + " must be greater than 0. The current setting is " + numThreads + + ". Reset it to the default value " + + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE); + numThreads = + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE; + } + this.initThreadsNumForStripedReads(numThreads); this.saslClient = new SaslDataTransferClient( conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf), TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth); @@ -3193,11 +3207,52 @@ public void rejectedExecution(Runnable runnable, LOG.debug("Using hedged reads; pool threads=" + num); } } + + /** + * Create thread pool for parallel reading in striped layout, + * STRIPED_READ_THREAD_POOL, if it does not already exist. + * @param num Number of threads for striped reads thread pool. + */ + private void initThreadsNumForStripedReads(int num) { + assert num > 0; + if (STRIPED_READ_THREAD_POOL != null) { + return; + } + synchronized (DFSClient.class) { + if (STRIPED_READ_THREAD_POOL == null) { + STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60, + TimeUnit.SECONDS, new SynchronousQueue(), + new Daemon.DaemonFactory() { + private final AtomicInteger threadIndex = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable r) { + Thread t = super.newThread(r); + t.setName("stripedRead-" + threadIndex.getAndIncrement()); + return t; + } + }, new ThreadPoolExecutor.CallerRunsPolicy() { + @Override + public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) { + LOG.info("Execution for striped reading rejected, " + + "Executing in current thread"); + // will run in the current thread + super.rejectedExecution(runnable, e); + } + }); + STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true); + } + } + } ThreadPoolExecutor getHedgedReadsThreadPool() { return HEDGED_READ_THREAD_POOL; } + ThreadPoolExecutor getStripedReadsThreadPool() { + return STRIPED_READ_THREAD_POOL; + } + boolean isHedgedReadsEnabled() { return (HEDGED_READ_THREAD_POOL != null) && HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index bd86964a17c46..6bc005bd14b9f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -670,7 +670,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.namenode.reject-unresolved-dn-topology-mapping"; public static final boolean DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_DEFAULT = false; - + + public static final String DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE = + "dfs.client.striped.read.threadpool.size"; + // With default 3+2 schema, each normal read could span 3 DNs. So this + // default value accommodates 6 read streams + public static final int DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE = 18; + // Slow io warning log threshold settings for dfsclient and datanode. public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY = "dfs.datanode.slow.io.warning.threshold.ms"; 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 d1e0b9a5931e2..72725c43abe38 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 @@ -44,6 +44,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.base.Preconditions; import org.apache.commons.io.IOUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.ByteBufferReadable; @@ -94,7 +95,7 @@ public class DFSInputStream extends FSInputStream @VisibleForTesting public static boolean tcpReadsDisabledForTesting = false; private long hedgedReadOpsLoopNumForTesting = 0; - private final DFSClient dfsClient; + protected final DFSClient dfsClient; private AtomicBoolean closed = new AtomicBoolean(false); private final String src; private final boolean verifyChecksum; @@ -441,7 +442,7 @@ public List getAllBlocks() throws IOException { * @return located block * @throws IOException */ - private LocatedBlock getBlockAt(long offset) throws IOException { + protected LocatedBlock getBlockAt(long offset) throws IOException { synchronized(infoLock) { assert (locatedBlocks != null) : "locatedBlocks is null"; @@ -713,7 +714,7 @@ public synchronized int read() throws IOException { * Wraps different possible read implementations so that readBuffer can be * strategy-agnostic. */ - private interface ReaderStrategy { + interface ReaderStrategy { public int doRead(BlockReader blockReader, int off, int len) throws ChecksumException, IOException; } @@ -1058,7 +1059,7 @@ private static String getBestNodeDNAddrPairErrorString( return errMsgr.toString(); } - private void fetchBlockByteRange(long blockStartOffset, long start, long end, + protected void fetchBlockByteRange(long blockStartOffset, long start, long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { @@ -1100,13 +1101,42 @@ public ByteBuffer call() throws Exception { }; } + /** + * Used when reading contiguous blocks + */ private void actualGetFromOneDataNode(final DNAddrPair datanode, long blockStartOffset, final long start, final long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { + final int length = (int) (end - start + 1); + actualGetFromOneDataNode(datanode, block, start, end, buf, + new int[]{offset}, new int[]{length}, corruptedBlockMap); + } + + /** + * Read data from one DataNode. + * @param datanode the datanode from which to read data + * @param block the block to read + * @param startInBlk the startInBlk offset of the block + * @param endInBlk the endInBlk offset of the block + * @param buf the given byte array into which the data is read + * @param offsets the data may be read into multiple segments of the buf + * (when reading a striped block). this array indicates the + * offset of each buf segment. + * @param lengths the length of each buf segment + * @param corruptedBlockMap map recording list of datanodes with corrupted + * block replica + */ + void actualGetFromOneDataNode(final DNAddrPair datanode, + LocatedBlock block, final long startInBlk, final long endInBlk, + byte[] buf, int[] offsets, int[] lengths, + Map> corruptedBlockMap) + throws IOException { DFSClientFaultInjector.get().startFetchFromDatanode(); int refetchToken = 1; // only need to get a new access token once int refetchEncryptionKey = 1; // only need to get a new encryption key once + final int len = (int) (endInBlk - startInBlk + 1); + checkReadPortions(offsets, lengths, len); while (true) { // cached block locations may have been updated by chooseDataNode() @@ -1116,15 +1146,15 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, BlockReader reader = null; try { DFSClientFaultInjector.get().fetchFromDatanodeException(); - int len = (int) (end - start + 1); reader = getBlockReader(block, start, len, datanode.addr, datanode.storageType, datanode.info); - int nread = reader.readAll(buf, offset, len); - updateReadStatistics(readStatistics, nread, reader); - - if (nread != len) { - throw new IOException("truncated return from reader.read(): " + - "excpected " + len + ", got " + nread); + for (int i = 0; i < offsets.length; i++) { + int nread = reader.readAll(buf, offsets[i], lengths[i]); + updateReadStatistics(readStatistics, nread, reader); + if (nread != len) { + throw new IOException("truncated return from reader.read(): " + + "excpected " + len + ", got " + nread); + } } DFSClientFaultInjector.get().readFromDatanodeDelay(); return; @@ -1169,7 +1199,26 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, } /** - * Like {@link #fetchBlockByteRange} except we start up a second, parallel, + * This method verifies that the read portions are valid and do not overlap + * with each other. + */ + private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) { + Preconditions.checkArgument(offsets.length == lengths.length && + offsets.length > 0); + int sum = 0; + for (int i = 0; i < lengths.length; i++) { + if (i > 0) { + int gap = offsets[i] - offsets[i - 1]; + // make sure read portions do not overlap with each other + Preconditions.checkArgument(gap >= lengths[i - 1]); + } + sum += lengths[i]; + } + Preconditions.checkArgument(sum == totalLen); + } + + /** + * Like {@link #fetchBlockByteRange}except we start up a second, parallel, * 'hedged' read if the first read is taking longer than configured amount of * time. We then wait on which ever read returns first. */ @@ -1388,10 +1437,9 @@ private int pread(long position, byte[] buffer, int offset, int length) long targetStart = position - blk.getStartOffset(); long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart); try { - if (dfsClient.isHedgedReadsEnabled()) { + if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) { hedgedFetchBlockByteRange(blk.getStartOffset(), targetStart, - targetStart + bytesToRead - 1, buffer, offset, - corruptedBlockMap); + targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap); } else { fetchBlockByteRange(blk.getStartOffset(), targetStart, targetStart + bytesToRead - 1, buffer, offset, @@ -1587,7 +1635,7 @@ public void reset() throws IOException { } /** Utility class to encapsulate data node info and its address. */ - private static final class DNAddrPair { + static final class DNAddrPair { final DatanodeInfo info; final InetSocketAddress addr; final StorageType storageType; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java new file mode 100644 index 0000000000000..077b0f855428e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -0,0 +1,367 @@ +/** + * 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 com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; +import org.apache.hadoop.net.NetUtils; +import org.apache.htrace.Span; +import org.apache.htrace.Trace; +import org.apache.htrace.TraceScope; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.CancellationException; +import java.util.concurrent.Callable; +import java.util.concurrent.Future; + + +/****************************************************************************** + * DFSStripedInputStream reads from striped block groups, illustrated below: + * + * | <- Striped Block Group -> | + * blk_0 blk_1 blk_2 <- A striped block group has + * | | | {@link #groupSize} blocks + * v v v + * +------+ +------+ +------+ + * |cell_0| |cell_1| |cell_2| <- The logical read order should be + * +------+ +------+ +------+ cell_0, cell_1, ... + * |cell_3| |cell_4| |cell_5| + * +------+ +------+ +------+ + * |cell_6| |cell_7| |cell_8| + * +------+ +------+ +------+ + * |cell_9| + * +------+ <- A cell contains {@link #cellSize} bytes of data + * + * Three styles of read will eventually be supported: + * 1. Stateful read: TODO: HDFS-8033 + * 2. pread without decode support + * This is implemented by calculating the portion of read from each block and + * issuing requests to each DataNode in parallel. + * 3. pread with decode support: TODO: will be supported after HDFS-7678 + *****************************************************************************/ +public class DFSStripedInputStream extends DFSInputStream { + /** + * This method plans the read portion from each block in the stripe + * @param groupSize The size / width of the striping group + * @param cellSize The size of each striping cell + * @param startInBlk Starting offset in the striped block + * @param len Length of the read request + * @param bufOffset Initial offset in the result buffer + * @return array of {@link ReadPortion}, each representing the portion of I/O + * for an individual block in the group + */ + @VisibleForTesting + static ReadPortion[] planReadPortions(final int groupSize, + final int cellSize, final long startInBlk, final int len, int bufOffset) { + ReadPortion[] results = new ReadPortion[groupSize]; + for (int i = 0; i < groupSize; i++) { + results[i] = new ReadPortion(); + } + + // cellIdxInBlk is the index of the cell in the block + // E.g., cell_3 is the 2nd cell in blk_0 + int cellIdxInBlk = (int) (startInBlk / (cellSize * groupSize)); + + // blkIdxInGroup is the index of the block in the striped block group + // E.g., blk_2 is the 3rd block in the group + final int blkIdxInGroup = (int) (startInBlk / cellSize % groupSize); + results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk + + startInBlk % cellSize; + boolean crossStripe = false; + for (int i = 1; i < groupSize; i++) { + if (blkIdxInGroup + i >= groupSize && !crossStripe) { + cellIdxInBlk++; + crossStripe = true; + } + results[(blkIdxInGroup + i) % groupSize].startOffsetInBlock = + cellSize * cellIdxInBlk; + } + + int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len); + results[blkIdxInGroup].offsetsInBuf.add(bufOffset); + results[blkIdxInGroup].lengths.add(firstCellLen); + results[blkIdxInGroup].readLength += firstCellLen; + + int i = (blkIdxInGroup + 1) % groupSize; + for (int done = firstCellLen; done < len; done += cellSize) { + ReadPortion rp = results[i]; + rp.offsetsInBuf.add(done + bufOffset); + final int readLen = Math.min(len - done, cellSize); + rp.lengths.add(readLen); + rp.readLength += readLen; + i = (i + 1) % groupSize; + } + return results; + } + + /** + * This method parses a striped block group into individual blocks. + * + * @param bg The striped block group + * @param dataBlkNum the number of data blocks + * @return An array containing the blocks in the group + */ + @VisibleForTesting + static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, + int dataBlkNum, int cellSize) { + int locatedBGSize = bg.getBlockIndices().length; + // TODO not considering missing blocks for now, only identify data blocks + LocatedBlock[] lbs = new LocatedBlock[dataBlkNum]; + for (short i = 0; i < locatedBGSize; i++) { + final int idx = bg.getBlockIndices()[i]; + if (idx < dataBlkNum && lbs[idx] == null) { + lbs[idx] = constructInternalBlock(bg, i, cellSize, idx); + } + } + return lbs; + } + + private static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, + int idxInReturnedLocs, int cellSize, int idxInBlockGroup) { + final ExtendedBlock blk = new ExtendedBlock(bg.getBlock()); + blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup); + // TODO: fix the numBytes computation + + return new LocatedBlock(blk, + new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, + new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, + new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, + bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), + null); + } + + + private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS; + + DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum) + throws IOException { + super(dfsClient, src, verifyChecksum); + DFSClient.LOG.debug("Creating an striped input stream for file " + src); + } + + @Override + public synchronized int read(final ByteBuffer buf) throws IOException { + throw new UnsupportedActionException("Stateful read is not supported"); + } + + @Override + public synchronized int read(final byte buf[], int off, int len) + throws IOException { + throw new UnsupportedActionException("Stateful read is not supported"); + } + + /** + * | <--------- LocatedStripedBlock (ID = 0) ---------> | + * LocatedBlock (0) | LocatedBlock (1) | LocatedBlock (2) + * ^ + * offset + * On a striped file, the super method {@link DFSInputStream#getBlockAt} + * treats a striped block group as a single {@link LocatedBlock} object, + * which includes target in its range. This method adds the logic of: + * 1. Analyzing the index of required block based on offset + * 2. Parsing the block group to obtain the block location on that index + */ + @Override + protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { + LocatedBlock lb = super.getBlockAt(blkStartOffset); + assert lb instanceof LocatedStripedBlock : "NameNode should return a " + + "LocatedStripedBlock for a striped file"; + + int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize) + % groupSize); + // If indexing information is returned, iterate through the index array + // to find the entry for position idx in the group + LocatedStripedBlock lsb = (LocatedStripedBlock) lb; + int i = 0; + for (; i < lsb.getBlockIndices().length; i++) { + if (lsb.getBlockIndices()[i] == idx) { + break; + } + } + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("getBlockAt for striped blocks, offset=" + + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx); + } + return constructInternalBlock(lsb, i, cellSize, idx); + } + + private LocatedBlock getBlockGroupAt(long offset) throws IOException { + return super.getBlockAt(offset); + } + + /** + * Real implementation of pread. + */ + @Override + protected void fetchBlockByteRange(LocatedBlock block, long start, + long end, byte[] buf, int offset, + Map> corruptedBlockMap) + throws IOException { + Map, Integer> futures = new HashMap<>(); + CompletionService stripedReadsService = + new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); + int len = (int) (end - start + 1); + + // Refresh the striped block group + block = getBlockGroupAt(block.getStartOffset()); + assert block instanceof LocatedStripedBlock : "NameNode" + + " should return a LocatedStripedBlock for a striped file"; + LocatedStripedBlock blockGroup = (LocatedStripedBlock) block; + + // Planning the portion of I/O for each shard + ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, start, + len, offset); + + // Parse group to get chosen DN location + LocatedBlock[] blks = parseStripedBlockGroup(blockGroup, groupSize, cellSize); + + for (short i = 0; i < groupSize; i++) { + ReadPortion rp = readPortions[i]; + if (rp.readLength <= 0) { + continue; + } + DatanodeInfo loc = blks[i].getLocations()[0]; + StorageType type = blks[i].getStorageTypes()[0]; + DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( + loc.getXferAddr(dfsClient.getConf().connectToDnViaHostname)), type); + Callable readCallable = getFromOneDataNode(dnAddr, blks[i], + rp.startOffsetInBlock, rp.startOffsetInBlock + rp.readLength - 1, buf, + rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i); + Future getFromDNRequest = stripedReadsService.submit(readCallable); + DFSClient.LOG.debug("Submitting striped read request for " + blks[i]); + futures.put(getFromDNRequest, (int) i); + } + while (!futures.isEmpty()) { + try { + waitNextCompletion(stripedReadsService, futures); + } catch (InterruptedException ie) { + // Ignore and retry + } + } + } + + private Callable getFromOneDataNode(final DNAddrPair datanode, + final LocatedBlock block, final long start, final long end, + final byte[] buf, final int[] offsets, final int[] lengths, + final Map> corruptedBlockMap, + final int hedgedReadId) { + final Span parentSpan = Trace.currentSpan(); + return new Callable() { + @Override + public Void call() throws Exception { + TraceScope scope = + Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan); + try { + actualGetFromOneDataNode(datanode, block, start, + end, buf, offsets, lengths, corruptedBlockMap); + } finally { + scope.close(); + } + return null; + } + }; + } + + private void waitNextCompletion(CompletionService stripedReadsService, + Map, Integer> futures) throws InterruptedException { + if (futures.isEmpty()) { + throw new InterruptedException("Futures already empty"); + } + Future future = null; + try { + future = stripedReadsService.take(); + future.get(); + futures.remove(future); + } catch (ExecutionException | CancellationException e) { + // already logged in the Callable + futures.remove(future); + } + throw new InterruptedException("let's retry"); + } + + public void setCellSize(int cellSize) { + this.cellSize = cellSize; + } + + /** + * This class represents the portion of I/O associated with each block in the + * striped block group. + */ + static class ReadPortion { + /** + * startOffsetInBlock + * | + * v + * |<-lengths[0]->|<- lengths[1] ->|<-lengths[2]->| + * +------------------+------------------+----------------+ + * | cell_0 | cell_3 | cell_6 | <- blk_0 + * +------------------+------------------+----------------+ + * _/ \_______________________ + * | | + * v offsetsInBuf[0] v offsetsInBuf[1] + * +------------------------------------------------------+ + * | cell_0 | cell_1 and cell_2 |cell_3 ...| <- buf + * | (partial) | (from blk_1 and blk_2) | | + * +------------------------------------------------------+ + */ + private long startOffsetInBlock = 0; + private long readLength = 0; + private final List offsetsInBuf = new ArrayList<>(); + private final List lengths = new ArrayList<>(); + + int[] getOffsets() { + int[] offsets = new int[offsetsInBuf.size()]; + for (int i = 0; i < offsets.length; i++) { + offsets[i] = offsetsInBuf.get(i); + } + return offsets; + } + + int[] getLengths() { + int[] lens = new int[this.lengths.size()]; + for (int i = 0; i < lens.length; i++) { + lens[i] = this.lengths.get(i); + } + return lens; + } + + long getReadLength() { + return readLength; + } + + long getStartOffsetInBlock() { + return startOffsetInBlock; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java index 97e3a6936f840..98614db446538 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -65,4 +65,9 @@ public String toString() { public int[] getBlockIndices() { return this.blockIndices; } + + @Override + public boolean isStriped() { + return true; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 4a85efbcbcd1d..20b0c5c4a27c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -23,7 +23,7 @@ import java.io.DataOutput; import java.io.IOException; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; /** * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. @@ -203,8 +203,8 @@ public long spaceConsumed() { // In case striped blocks, total usage by this striped blocks should // be the total of data blocks and parity blocks because // `getNumBytes` is the total of actual data block size. - return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CHUNK_SIZE) + 1) - * BLOCK_STRIPED_CHUNK_SIZE * parityBlockNum + getNumBytes(); + return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CELL_SIZE) + 1) + * BLOCK_STRIPED_CELL_SIZE * parityBlockNum + getNumBytes(); } @Override 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 53a99b05eb931..24943816bd306 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 @@ -66,6 +66,12 @@ import java.util.UUID; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.base.Charsets; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; @@ -102,6 +108,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LayoutVersion; +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.protocol.datatransfer.Sender; @@ -124,8 +131,10 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; +import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSEditLog; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.LeaseManager; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider; @@ -134,7 +143,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; -import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.io.IOUtils; @@ -156,12 +164,8 @@ import org.mockito.internal.util.reflection.Whitebox; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; /** Utilities for HDFS tests */ public class DFSTestUtil { @@ -1846,4 +1850,77 @@ public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock( reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks); return reports; } + + public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, + int numBlocks, int numStripesPerBlk) throws Exception { + DistributedFileSystem dfs = cluster.getFileSystem(); + dfs.mkdirs(dir); + dfs.getClient().createErasureCodingZone(dir.toString()); + + FSDataOutputStream out = null; + try { + out = dfs.create(file, (short) 1); // create an empty file + + FSNamesystem ns = cluster.getNamesystem(); + FSDirectory fsdir = ns.getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + + ExtendedBlock previous = null; + for (int i = 0; i < numBlocks; i++) { + Block newBlock = createBlock(cluster.getDataNodes(), dfs, ns, + file.toString(), fileNode, dfs.getClient().getClientName(), + previous, numStripesPerBlk); + previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock); + } + + dfs.getClient().namenode.complete(file.toString(), + dfs.getClient().getClientName(), previous, fileNode.getId()); + } finally { + IOUtils.cleanup(null, out); + } + } + + static Block createBlock(List dataNodes, DistributedFileSystem fs, + FSNamesystem ns, String file, INodeFile fileNode, String clientName, + ExtendedBlock previous, int numStripes) throws Exception { + fs.getClient().namenode.addBlock(file, clientName, previous, null, + fileNode.getId(), null); + + final BlockInfo lastBlock = fileNode.getLastBlock(); + final int groupSize = fileNode.getBlockReplication(); + // 1. RECEIVING_BLOCK IBR + int i = 0; + for (DataNode dn : dataNodes) { + if (i < groupSize) { + final Block block = new Block(lastBlock.getBlockId() + i++, 0, + lastBlock.getGenerationStamp()); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage); + for (StorageReceivedDeletedBlocks report : reports) { + ns.processIncrementalBlockReport(dn.getDatanodeId(), report); + } + } + } + + // 2. RECEIVED_BLOCK IBR + i = 0; + for (DataNode dn : dataNodes) { + if (i < groupSize) { + final Block block = new Block(lastBlock.getBlockId() + i++, + numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp()); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + for (StorageReceivedDeletedBlocks report : reports) { + ns.processIncrementalBlockReport(dn.getDatanodeId(), report); + } + } + } + + lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS); + return lastBlock; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java new file mode 100644 index 0000000000000..0032bdd47dc47 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java @@ -0,0 +1,304 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +public class TestReadStripedFile { + + public static final Log LOG = LogFactory.getLog(TestReadStripedFile.class); + + private MiniDFSCluster cluster; + private Configuration conf = new Configuration(); + private DistributedFileSystem fs; + private final Path dirPath = new Path("/striped"); + private Path filePath = new Path(dirPath, "file"); + private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS; + private final short TOTAL_SIZE = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; + private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int NUM_STRIPE_PER_BLOCK = 2; + private final int BLOCKSIZE = 2 * GROUP_SIZE * CELLSIZE; + + @Before + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE); + SimulatedFSDataset.setFactory(conf); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(TOTAL_SIZE) + .build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + private void testPlanReadPortions(int startInBlk, int length, + int bufferOffset, int[] readLengths, int[] offsetsInBlock, + int[][] bufferOffsets, int[][] bufferLengths) { + ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE, + CELLSIZE, startInBlk, length, bufferOffset); + assertEquals(GROUP_SIZE, results.length); + + for (int i = 0; i < GROUP_SIZE; i++) { + assertEquals(readLengths[i], results[i].getReadLength()); + assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock()); + final int[] bOffsets = results[i].getOffsets(); + assertArrayEquals(bufferOffsets[i], bOffsets); + final int[] bLengths = results[i].getLengths(); + assertArrayEquals(bufferLengths[i], bLengths); + } + } + + /** + * Test {@link DFSStripedInputStream#planReadPortions} + */ + @Test + public void testPlanReadPortions() { + /** + * start block offset is 0, read cellSize - 10 + */ + testPlanReadPortions(0, CELLSIZE - 10, 0, + new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0}, + new int[][]{new int[]{0}, new int[]{}, new int[]{}}, + new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}}); + + /** + * start block offset is 0, read 3 * cellSize + */ + testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0, + new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0}, + new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}}, + new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}}); + + /** + * start block offset is 0, read cellSize + 10 + */ + testPlanReadPortions(0, CELLSIZE + 10, 0, + new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0}, + new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}}, + new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}}); + + /** + * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100 + */ + testPlanReadPortions(0, 5 * CELLSIZE + 10, 100, + new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0}, + new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE}, + new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4}, + new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}}, + new int[][]{new int[]{CELLSIZE, CELLSIZE}, + new int[]{CELLSIZE, CELLSIZE}, + new int[]{CELLSIZE, 10}}); + + /** + * start block offset is 2, read 3 * cellSize + */ + testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100, + new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, + new int[]{2, 0, 0}, + new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2}, + new int[]{100 + CELLSIZE - 2}, + new int[]{100 + CELLSIZE * 2 - 2}}, + new int[][]{new int[]{CELLSIZE - 2, 2}, + new int[]{CELLSIZE}, + new int[]{CELLSIZE}}); + + /** + * start block offset is 2, read 3 * cellSize + 10 + */ + testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0, + new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE}, + new int[]{2, 0, 0}, + new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2}, + new int[]{CELLSIZE - 2}, + new int[]{CELLSIZE * 2 - 2}}, + new int[][]{new int[]{CELLSIZE - 2, 12}, + new int[]{CELLSIZE}, + new int[]{CELLSIZE}}); + + /** + * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10 + */ + testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0, + new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2}, + new int[]{CELLSIZE, CELLSIZE - 1, 0}, + new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1}, + new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}, + new int[]{1, 3 * CELLSIZE + 1}}, + new int[][]{new int[]{CELLSIZE, CELLSIZE}, + new int[]{1, CELLSIZE, 9}, + new int[]{CELLSIZE, CELLSIZE}}); + + /** + * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10 + */ + testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0, + new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1}, + new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1}, + new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1}, + new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1}, + new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}}, + new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, + new int[]{CELLSIZE, CELLSIZE, 9}, + new int[]{1, CELLSIZE, CELLSIZE}}); + } + + private LocatedStripedBlock createDummyLocatedBlock() { + final long blockGroupID = -1048576; + DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE]; + String[] storageIDs = new String[TOTAL_SIZE]; + StorageType[] storageTypes = new StorageType[TOTAL_SIZE]; + int[] indices = new int[TOTAL_SIZE]; + for (int i = 0; i < TOTAL_SIZE; i++) { + locs[i] = new DatanodeInfo(cluster.getDataNodes().get(i).getDatanodeId()); + storageIDs[i] = cluster.getDataNodes().get(i).getDatanodeUuid(); + storageTypes[i] = StorageType.DISK; + indices[i] = (i + 2) % GROUP_SIZE; + } + return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID), + locs, storageIDs, storageTypes, indices, 0, false, null); + } + + @Test + public void testParseDummyStripedBlock() { + LocatedStripedBlock lsb = createDummyLocatedBlock(); + LocatedBlock[] blocks = DFSStripedInputStream.parseStripedBlockGroup( + lsb, GROUP_SIZE, CELLSIZE); + assertEquals(GROUP_SIZE, blocks.length); + for (int j = 0; j < GROUP_SIZE; j++) { + assertFalse(blocks[j].isStriped()); + assertEquals(j, + BlockIdManager.getBlockIndex(blocks[j].getBlock().getLocalBlock())); + assertEquals(j * CELLSIZE, blocks[j].getStartOffset()); + } + } + + @Test + public void testParseStripedBlock() throws Exception { + final int numBlocks = 4; + DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, + NUM_STRIPE_PER_BLOCK); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCKSIZE * numBlocks); + + assertEquals(4, lbs.locatedBlockCount()); + List lbList = lbs.getLocatedBlocks(); + for (LocatedBlock lb : lbList) { + assertTrue(lb.isStriped()); + } + + for (int i = 0; i < numBlocks; i++) { + LocatedStripedBlock lsb = (LocatedStripedBlock) (lbs.get(i)); + LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb, + GROUP_SIZE, CELLSIZE); + assertEquals(GROUP_SIZE, blks.length); + for (int j = 0; j < GROUP_SIZE; j++) { + assertFalse(blks[j].isStriped()); + assertEquals(j, + BlockIdManager.getBlockIndex(blks[j].getBlock().getLocalBlock())); + assertEquals(i * BLOCKSIZE + j * CELLSIZE, blks[j].getStartOffset()); + } + } + } + + /** + * Test {@link DFSStripedInputStream#getBlockAt(long)} + */ + @Test + public void testGetBlock() throws Exception { + final int numBlocks = 4; + DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, + NUM_STRIPE_PER_BLOCK); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCKSIZE * numBlocks); + final DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); + + List lbList = lbs.getLocatedBlocks(); + for (LocatedBlock aLbList : lbList) { + LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList; + LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb, + GROUP_SIZE, CELLSIZE); + for (int j = 0; j < GROUP_SIZE; j++) { + LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset()); + assertEquals(blks[j].getBlock(), refreshed.getBlock()); + assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset()); + assertArrayEquals(blks[j].getLocations(), refreshed.getLocations()); + } + } + } + + @Test + public void testPread() throws Exception { + final int numBlocks = 4; + DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, + NUM_STRIPE_PER_BLOCK); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCKSIZE); + + assert lbs.get(0) instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); + for (int i = 0; i < GROUP_SIZE; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, BLOCKSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); + } + DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); + in.setCellSize(CELLSIZE); + int readSize = BLOCKSIZE; + byte[] readBuffer = new byte[readSize]; + int ret = in.read(0, readBuffer, 0, readSize); + + assertEquals(readSize, ret); + // TODO: verify read results with patterned data from HDFS-8117 + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index d965ae721344c..b2ff6c881d3e7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -18,15 +18,11 @@ package org.apache.hadoop.hdfs.server.namenode; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; 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.protocol.Block; -import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; @@ -36,19 +32,14 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; -import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; -import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; -import org.apache.hadoop.io.IOUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; import java.io.IOException; import java.util.List; -import java.util.UUID; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -84,83 +75,10 @@ public void tearDown() throws Exception { } } - public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, - int numBlocks) throws Exception { - DistributedFileSystem dfs = cluster.getFileSystem(); - dfs.mkdirs(dir); - dfs.getClient().getNamenode().createErasureCodingZone(dir.toString()); - - FSDataOutputStream out = null; - try { - out = dfs.create(file, (short) 1); // create an empty file - - FSNamesystem ns = cluster.getNamesystem(); - FSDirectory fsdir = ns.getFSDirectory(); - INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); - - ExtendedBlock previous = null; - for (int i = 0; i < numBlocks; i++) { - Block newBlock = createBlock(cluster.getDataNodes(), ns, - file.toString(), fileNode, dfs.getClient().getClientName(), - previous); - previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock); - } - - ns.completeFile(file.toString(), dfs.getClient().getClientName(), - previous, fileNode.getId()); - } finally { - IOUtils.cleanup(null, out); - } - } - - static Block createBlock(List dataNodes, FSNamesystem ns, - String file, INodeFile fileNode, String clientName, - ExtendedBlock previous) throws Exception { - ns.getAdditionalBlock(file, fileNode.getId(), clientName, previous, null, - null); - - final BlockInfo lastBlock = fileNode.getLastBlock(); - final int groupSize = fileNode.getBlockReplication(); - // 1. RECEIVING_BLOCK IBR - int i = 0; - for (DataNode dn : dataNodes) { - if (i < groupSize) { - final Block block = new Block(lastBlock.getBlockId() + i++, 0, - lastBlock.getGenerationStamp()); - DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); - StorageReceivedDeletedBlocks[] reports = DFSTestUtil - .makeReportForReceivedBlock(block, - ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage); - for (StorageReceivedDeletedBlocks report : reports) { - ns.processIncrementalBlockReport(dn.getDatanodeId(), report); - } - } - } - - // 2. RECEIVED_BLOCK IBR - i = 0; - for (DataNode dn : dataNodes) { - if (i < groupSize) { - final Block block = new Block(lastBlock.getBlockId() + i++, - BLOCK_STRIPED_CHUNK_SIZE, lastBlock.getGenerationStamp()); - DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); - StorageReceivedDeletedBlocks[] reports = DFSTestUtil - .makeReportForReceivedBlock(block, - ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); - for (StorageReceivedDeletedBlocks report : reports) { - ns.processIncrementalBlockReport(dn.getDatanodeId(), report); - } - } - } - - lastBlock.setNumBytes(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS); - return lastBlock; - } - @Test public void testMissingStripedBlock() throws Exception { final int numBlocks = 4; - createECFile(cluster, filePath, dirPath, numBlocks); + DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, 1); // make sure the file is complete in NN final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() @@ -172,7 +90,7 @@ public void testMissingStripedBlock() throws Exception { for (BlockInfo blk : blocks) { assertTrue(blk.isStriped()); assertTrue(blk.isComplete()); - assertEquals(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes()); + assertEquals(BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes()); final BlockInfoStriped sb = (BlockInfoStriped) blk; assertEquals(GROUP_SIZE, sb.numNodes()); } From bff59392eecbb6f62b4c92c80fee6b8fb6b555bb Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 7 Apr 2015 11:20:13 -0700 Subject: [PATCH 050/212] HDFS-7782. Erasure coding: pread from files in striped layout. Contributed by Zhe Zhang and Jing Zhao --- .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index a845fdfbf4395..ad92dfcc8ba0b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -3207,7 +3207,7 @@ public void rejectedExecution(Runnable runnable, LOG.debug("Using hedged reads; pool threads=" + num); } } - + /** * Create thread pool for parallel reading in striped layout, * STRIPED_READ_THREAD_POOL, if it does not already exist. From 16484f00308c8cdee1f4b26639587cfb171ea341 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 8 Apr 2015 12:48:59 +0530 Subject: [PATCH 051/212] HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +- .../org/apache/hadoop/hdfs/DFSClient.java | 14 ++++++ .../hadoop/hdfs/protocol/ClientProtocol.java | 10 ++++ ...amenodeProtocolServerSideTranslatorPB.java | 19 ++++++++ .../ClientNamenodeProtocolTranslatorPB.java | 18 ++++++++ .../hadoop/hdfs/protocolPB/PBHelper.java | 46 +++++++++++++++++++ .../hdfs/server/namenode/FSNamesystem.java | 31 +++++++++++++ .../server/namenode/NameNodeRpcServer.java | 7 +++ .../main/proto/ClientNamenodeProtocol.proto | 10 ++++ .../hadoop-hdfs/src/main/proto/hdfs.proto | 28 +++++++++++ .../hadoop/hdfs/TestErasureCodingZones.java | 38 ++++++++++++++- 11 files changed, 223 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9927ccff50aa3..74230337b7612 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -49,4 +49,7 @@ (Hui Zheng via Zhe Zhang) HDFS-7839. Erasure coding: implement facilities in NameNode to create and - manage EC zones (Zhe Zhang) \ No newline at end of file + manage EC zones (Zhe Zhang) + + HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from + NameNode (vinayakumarb) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index ad92dfcc8ba0b..c418c4d57fef3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -119,6 +119,7 @@ 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.ECInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -3135,6 +3136,19 @@ public void checkAccess(String src, FsAction mode) throws IOException { } } + public ECInfo getErasureCodingInfo(String src) throws IOException { + checkOpen(); + TraceScope scope = getPathTraceScope("getErasureCodingInfo", src); + try { + return namenode.getErasureCodingInfo(src); + } catch (RemoteException re) { + throw re.unwrapRemoteException(AccessControlException.class, + FileNotFoundException.class, UnresolvedPathException.class); + } finally { + scope.close(); + } + } + public DFSInotifyEventInputStream getInotifyEventStream() throws IOException { checkOpen(); return new DFSInotifyEventInputStream(traceSampler, namenode); 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 a94caad972a82..d27dac7c3d33e 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 @@ -1464,4 +1464,14 @@ public List listXAttrs(String src) */ @Idempotent public EventBatchList getEditsFromTxid(long txid) throws IOException; + + /** + * Gets the ECInfo for the specified file/directory + * + * @param src + * @return Returns the ECInfo if the file/directory is erasure coded, null otherwise + * @throws IOException + */ + @Idempotent + public ECInfo getErasureCodingInfo(String src) throws IOException; } 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 5be9bc6d8d7fc..4b110fe3a1851 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 @@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -108,6 +109,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; @@ -1511,4 +1514,20 @@ public GetEditsFromTxidResponseProto getEditsFromTxid(RpcController controller, throw new ServiceException(e); } } + + @Override + public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller, + GetErasureCodingInfoRequestProto request) throws ServiceException { + try { + ECInfo ecInfo = server.getErasureCodingInfo(request.getSrc()); + GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto + .newBuilder(); + if (ecInfo != null) { + resBuilder.setECInfo(PBHelper.convertECInfo(ecInfo)); + } + return resBuilder.build(); + } 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 43a0322c25c5b..568da687436ef 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 @@ -58,6 +58,7 @@ 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.ECInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -107,6 +108,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; @@ -1532,4 +1535,19 @@ public EventBatchList getEditsFromTxid(long txid) throws IOException { throw ProtobufHelper.getRemoteException(e); } } + + @Override + public ECInfo getErasureCodingInfo(String src) throws IOException { + GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder() + .setSrc(src).build(); + try { + GetErasureCodingInfoResponseProto res = rpcProxy.getErasureCodingInfo(null, req); + if (res.hasECInfo()) { + return PBHelper.convertECInfo(res.getECInfo()); + } + return null; + } 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 635c8230d8150..c0a482cd4e29e 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 @@ -29,7 +29,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; import org.apache.hadoop.fs.CacheFlag; import org.apache.hadoop.fs.ContentSummary; @@ -77,6 +81,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -146,6 +151,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto; @@ -226,6 +234,7 @@ import org.apache.hadoop.hdfs.util.ExactSizeInputStream; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; @@ -3095,4 +3104,41 @@ public static BlockReportContextProto convert(BlockReportContext context) { setId(context.getReportId()). build(); } + + public static ECInfo convertECInfo(ECInfoProto ecInfoProto) { + return new ECInfo(ecInfoProto.getSrc(), + convertECSchema(ecInfoProto.getSchema())); + } + + public static ECInfoProto convertECInfo(ECInfo ecInfo) { + return ECInfoProto.newBuilder().setSrc(ecInfo.getSrc()) + .setSchema(convertECSchema(ecInfo.getSchema())).build(); + } + + public static ECSchema convertECSchema(ECSchemaProto schema) { + List optionsList = schema.getOptionsList(); + Map options = new HashMap<>(optionsList.size()); + for (ECSchemaOptionEntryProto option : optionsList) { + options.put(option.getKey(), option.getValue()); + } + // include chunksize in options. + options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(schema.getChunkSize())); + return new ECSchema(schema.getSchemaName(), schema.getCodecName(), + schema.getDataUnits(), schema.getParityUnits(), options); + } + + public static ECSchemaProto convertECSchema(ECSchema schema) { + ECSchemaProto.Builder builder = ECSchemaProto.newBuilder() + .setSchemaName(schema.getSchemaName()) + .setCodecName(schema.getCodecName()) + .setDataUnits(schema.getNumDataUnits()) + .setParityUnits(schema.getNumParityUnits()) + .setChunkSize(schema.getChunkSize()); + Set> entrySet = schema.getOptions().entrySet(); + for (Entry entry : entrySet) { + builder.addOptions(ECSchemaOptionEntryProto.newBuilder() + .setKey(entry.getKey()).setValue(entry.getValue()).build()); + } + return builder.build(); + } } 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 d19a1db817516..c2bca5e199de2 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 @@ -180,6 +180,7 @@ 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.ECInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -262,6 +263,7 @@ import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.RetriableException; import org.apache.hadoop.ipc.RetryCache; import org.apache.hadoop.ipc.Server; @@ -7554,6 +7556,35 @@ void createErasureCodingZone(final String srcArg, logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat); } + /** + * Get the erasure coding information for specified src + */ + ECInfo getErasureCodingInfo(String src) throws AccessControlException, + UnresolvedLinkException, IOException { + checkOperation(OperationCategory.READ); + final byte[][] pathComponents = FSDirectory + .getPathComponentsForReservedPath(src); + final FSPermissionChecker pc = getPermissionChecker(); + readLock(); + try { + checkOperation(OperationCategory.READ); + src = dir.resolvePath(pc, src, pathComponents); + final INodesInPath iip = dir.getINodesInPath(src, true); + if (isPermissionEnabled) { + dir.checkPathAccess(pc, iip, FsAction.READ); + } + if (dir.getECPolicy(iip)) { + // TODO HDFS-8074 and HDFS-7859 : To get from loaded schemas + Map options = new HashMap(); + ECSchema defaultSchema = new ECSchema("RS-6-3", "rs", 6, 3, options); + return new ECInfo(src, defaultSchema); + } + } finally { + readUnlock(); + } + return null; + } + void setXAttr(String src, XAttr xAttr, EnumSet flag, boolean logRetryCache) throws IOException { 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 6e0333cc76bc9..2d19e47f5acce 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 @@ -84,6 +84,7 @@ 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.ECInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FSLimitException; @@ -2029,4 +2030,10 @@ public void removeSpanReceiver(long id) throws IOException { namesystem.checkSuperuserPrivilege(); nn.spanReceiverHost.removeSpanReceiver(id); } + + @Override // ClientNameNodeProtocol + public ECInfo getErasureCodingInfo(String src) throws IOException { + checkNNStartup(); + return namesystem.getErasureCodingInfo(src); + } } 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 183aff89a41a4..9488aed8c17ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -721,6 +721,14 @@ message CreateErasureCodingZoneRequestProto { message CreateErasureCodingZoneResponseProto { } +message GetErasureCodingInfoRequestProto { + required string src = 1; +} + +message GetErasureCodingInfoResponseProto { + optional ECInfoProto ECInfo = 1; +} + service ClientNamenodeProtocol { rpc getBlockLocations(GetBlockLocationsRequestProto) returns(GetBlockLocationsResponseProto); @@ -869,4 +877,6 @@ service ClientNamenodeProtocol { returns(GetCurrentEditLogTxidResponseProto); rpc getEditsFromTxid(GetEditsFromTxidRequestProto) returns(GetEditsFromTxidResponseProto); + rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto) + returns(GetErasureCodingInfoResponseProto); } 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 67e20582f2230..1314ea0718311 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -620,3 +620,31 @@ message RollingUpgradeStatusProto { required string blockPoolId = 1; optional bool finalized = 2 [default = false]; } + +/** + * ECSchema options entry + */ +message ECSchemaOptionEntryProto { + required string key = 1; + required string value = 2; +} + +/** + * ECShema for erasurecoding + */ +message ECSchemaProto { + required string schemaName = 1; + required string codecName = 2; + required uint32 dataUnits = 3; + required uint32 parityUnits = 4; + required uint32 chunkSize = 5; + repeated ECSchemaOptionEntryProto options = 6; +} + +/** + * ECInfo + */ +message ECInfoProto { + required string src = 1; + required ECSchemaProto schema = 2; +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index 49f08eefdf6f0..bdca915b2145d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -20,8 +20,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INode; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -29,8 +31,7 @@ import java.io.IOException; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; public class TestErasureCodingZones { private final int NUM_OF_DATANODES = 3; @@ -148,4 +149,37 @@ public void testMoveValidity() throws IOException, InterruptedException { "destination have different erasure coding policies", e); } } + + @Test + public void testGetErasureCodingInfo() throws Exception { + String src = "/ec"; + final Path ecDir = new Path(src); + fs.mkdir(ecDir, FsPermission.getDirDefault()); + // dir ECInfo before creating ec zone + assertNull(fs.getClient().getErasureCodingInfo(src)); + // dir ECInfo after creating ec zone + fs.getClient().createErasureCodingZone(src); + verifyErasureCodingInfo(src); + fs.create(new Path(ecDir, "/child1")).close(); + // verify for the files in ec zone + verifyErasureCodingInfo(src + "/child1"); + } + + private void verifyErasureCodingInfo(String src) throws IOException { + ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src); + assertNotNull("ECInfo should have been non-null", ecInfo); + assertEquals(src, ecInfo.getSrc()); + ECSchema schema = ecInfo.getSchema(); + assertNotNull(schema); + assertEquals("Default schema should be returned", "RS-6-3", + schema.getSchemaName()); + assertEquals("Default codec(rs) should be returned", "rs", + schema.getCodecName()); + assertEquals("Default numDataUnits should be used", 6, + schema.getNumDataUnits()); + assertEquals("Default numParityUnits should be used", 3, + schema.getNumParityUnits()); + assertEquals("Default chunkSize should be used", + ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize()); + } } \ No newline at end of file From 2ca3e38feb4191ef17fd58b8b4719fd5297a5e21 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 8 Apr 2015 14:23:03 +0530 Subject: [PATCH 052/212] HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B) Added missed file --- .../apache/hadoop/hdfs/protocol/ECInfo.java | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java new file mode 100644 index 0000000000000..ca642c233777f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * Class to provide information, such as ECSchema, for a file/block. + */ +public class ECInfo { + private final String src; + private final ECSchema schema; + + public ECInfo(String src, ECSchema schema) { + this.src = src; + this.schema = schema; + } + + public String getSrc() { + return src; + } + + public ECSchema getSchema() { + return schema; + } +} From 1a37b0fe9aaf30e775de63c538aeac82fe378400 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Thu, 9 Apr 2015 01:30:02 +0800 Subject: [PATCH 053/212] HDFS-8074 Define a system-wide default EC schema. Contributed by Kai Zheng --- .../src/main/conf/ecschema-def.xml | 5 -- .../hadoop/io/erasurecode/ECSchema.java | 57 ++++++++++++++++- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 4 +- .../hdfs/server/namenode/ECSchemaManager.java | 62 +++++++++++++++++++ 4 files changed, 120 insertions(+), 8 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml index e6194857ef2e3..e36d38650e443 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml +++ b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml @@ -27,11 +27,6 @@ You can modify and remove those not used yet, or add new ones. --> - - 6 - 3 - RS - 10 4 diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index 27be00e673277..8c3310e406717 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -23,12 +23,12 @@ /** * Erasure coding schema to housekeeper relevant information. */ -public class ECSchema { +public final class ECSchema { public static final String NUM_DATA_UNITS_KEY = "k"; public static final String NUM_PARITY_UNITS_KEY = "m"; public static final String CODEC_NAME_KEY = "codec"; public static final String CHUNK_SIZE_KEY = "chunkSize"; - public static final int DEFAULT_CHUNK_SIZE = 64 * 1024; // 64K + public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K private String schemaName; private String codecName; @@ -81,6 +81,18 @@ public ECSchema(String schemaName, Map options) { initWith(codecName, dataUnits, parityUnits, options); } + /** + * Constructor with key parameters provided. + * @param schemaName + * @param codecName + * @param numDataUnits + * @param numParityUnits + */ + public ECSchema(String schemaName, String codecName, + int numDataUnits, int numParityUnits) { + this(schemaName, codecName, numDataUnits, numParityUnits, null); + } + /** * Constructor with key parameters provided. Note the options may contain * additional information for the erasure codec to interpret further. @@ -200,4 +212,45 @@ public String toString() { return sb.toString(); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ECSchema ecSchema = (ECSchema) o; + + if (numDataUnits != ecSchema.numDataUnits) { + return false; + } + if (numParityUnits != ecSchema.numParityUnits) { + return false; + } + if (chunkSize != ecSchema.chunkSize) { + return false; + } + if (!schemaName.equals(ecSchema.schemaName)) { + return false; + } + if (!codecName.equals(ecSchema.codecName)) { + return false; + } + return options.equals(ecSchema.options); + } + + @Override + public int hashCode() { + int result = schemaName.hashCode(); + result = 31 * result + codecName.hashCode(); + result = 31 * result + options.hashCode(); + result = 31 * result + numDataUnits; + result = 31 * result + numParityUnits; + result = 31 * result + chunkSize; + + return result; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 74230337b7612..5078a1527473c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -52,4 +52,6 @@ manage EC zones (Zhe Zhang) HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from - NameNode (vinayakumarb) \ No newline at end of file + NameNode (vinayakumarb) + + HDFS-8074. Define a system-wide default EC schema. (Kai Zheng) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java new file mode 100644 index 0000000000000..b001c57fd2b27 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * This manages EC schemas predefined and activated in the system. It loads from + * predefined ones in XML and syncs with persisted ones in NameNode image. + * + * This class is instantiated by the FSNamesystem. + */ +@InterfaceAudience.LimitedPrivate({"HDFS"}) +public final class ECSchemaManager { + + private static final int DEFAULT_DATA_BLOCKS = 6; + private static final int DEFAULT_PARITY_BLOCKS = 3; + private static final String DEFAULT_CODEC_NAME = "rs"; + private static final String DEFAULT_SCHEMA_NAME = "SYS-DEFAULT-RS-6-3"; + + private static ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(DEFAULT_SCHEMA_NAME, + DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS); + + /** + * Get system-wide default EC schema, which can be used by default when no + * schema is specified for an EC zone. + * @return schema + */ + public static ECSchema getSystemDefaultSchema() { + return SYS_DEFAULT_SCHEMA; + } + + /** + * Tell the specified schema is the system default one or not. + * @param schema + * @return true if it's the default false otherwise + */ + public static boolean isSystemDefault(ECSchema schema) { + if (schema == null) { + throw new IllegalArgumentException("Invalid schema parameter"); + } + + // schema name is the identifier, but for safety we check all properties. + return SYS_DEFAULT_SCHEMA.equals(schema); + } +} From c4a0d88366aab16781ae132bb62a9c3aa2993b72 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 10 Apr 2015 00:16:28 +0800 Subject: [PATCH 054/212] HDFS-8104 Make hard-coded values consistent with the system default schema first before remove them. Contributed by Kai Zheng --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 4 +- .../hadoop/hdfs/TestPlanReadPortions.java | 142 ++++++++++++++++++ .../hadoop/hdfs/TestReadStripedFile.java | 112 -------------- 3 files changed, 145 insertions(+), 113 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 5078a1527473c..1e695c430e8f0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -54,4 +54,6 @@ HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (vinayakumarb) - HDFS-8074. Define a system-wide default EC schema. (Kai Zheng) \ No newline at end of file + HDFS-8074. Define a system-wide default EC schema. (Kai Zheng) + + HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java new file mode 100644 index 0000000000000..cf84b30340a94 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java @@ -0,0 +1,142 @@ +/** + * 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.junit.Test; + +import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion; +import static org.junit.Assert.*; + +public class TestPlanReadPortions { + + // We only support this as num of data blocks. It might be good enough for now + // for the purpose, even not flexible yet for any number in a schema. + private final short GROUP_SIZE = 3; + private final int CELLSIZE = 128 * 1024; + + private void testPlanReadPortions(int startInBlk, int length, + int bufferOffset, int[] readLengths, int[] offsetsInBlock, + int[][] bufferOffsets, int[][] bufferLengths) { + ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE, + CELLSIZE, startInBlk, length, bufferOffset); + assertEquals(GROUP_SIZE, results.length); + + for (int i = 0; i < GROUP_SIZE; i++) { + assertEquals(readLengths[i], results[i].getReadLength()); + assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock()); + final int[] bOffsets = results[i].getOffsets(); + assertArrayEquals(bufferOffsets[i], bOffsets); + final int[] bLengths = results[i].getLengths(); + assertArrayEquals(bufferLengths[i], bLengths); + } + } + + /** + * Test {@link DFSStripedInputStream#planReadPortions} + */ + @Test + public void testPlanReadPortions() { + /** + * start block offset is 0, read cellSize - 10 + */ + testPlanReadPortions(0, CELLSIZE - 10, 0, + new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0}, + new int[][]{new int[]{0}, new int[]{}, new int[]{}}, + new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}}); + + /** + * start block offset is 0, read 3 * cellSize + */ + testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0, + new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0}, + new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}}, + new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}}); + + /** + * start block offset is 0, read cellSize + 10 + */ + testPlanReadPortions(0, CELLSIZE + 10, 0, + new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0}, + new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}}, + new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}}); + + /** + * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100 + */ + testPlanReadPortions(0, 5 * CELLSIZE + 10, 100, + new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0}, + new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE}, + new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4}, + new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}}, + new int[][]{new int[]{CELLSIZE, CELLSIZE}, + new int[]{CELLSIZE, CELLSIZE}, + new int[]{CELLSIZE, 10}}); + + /** + * start block offset is 2, read 3 * cellSize + */ + testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100, + new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, + new int[]{2, 0, 0}, + new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2}, + new int[]{100 + CELLSIZE - 2}, + new int[]{100 + CELLSIZE * 2 - 2}}, + new int[][]{new int[]{CELLSIZE - 2, 2}, + new int[]{CELLSIZE}, + new int[]{CELLSIZE}}); + + /** + * start block offset is 2, read 3 * cellSize + 10 + */ + testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0, + new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE}, + new int[]{2, 0, 0}, + new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2}, + new int[]{CELLSIZE - 2}, + new int[]{CELLSIZE * 2 - 2}}, + new int[][]{new int[]{CELLSIZE - 2, 12}, + new int[]{CELLSIZE}, + new int[]{CELLSIZE}}); + + /** + * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10 + */ + testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0, + new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2}, + new int[]{CELLSIZE, CELLSIZE - 1, 0}, + new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1}, + new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}, + new int[]{1, 3 * CELLSIZE + 1}}, + new int[][]{new int[]{CELLSIZE, CELLSIZE}, + new int[]{1, CELLSIZE, 9}, + new int[]{CELLSIZE, CELLSIZE}}); + + /** + * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10 + */ + testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0, + new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1}, + new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1}, + new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1}, + new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1}, + new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}}, + new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, + new int[]{CELLSIZE, CELLSIZE, 9}, + new int[]{1, CELLSIZE, CELLSIZE}}); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java index 0032bdd47dc47..849e12ea8e49a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; -import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -78,117 +77,6 @@ public void tearDown() { } } - private void testPlanReadPortions(int startInBlk, int length, - int bufferOffset, int[] readLengths, int[] offsetsInBlock, - int[][] bufferOffsets, int[][] bufferLengths) { - ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE, - CELLSIZE, startInBlk, length, bufferOffset); - assertEquals(GROUP_SIZE, results.length); - - for (int i = 0; i < GROUP_SIZE; i++) { - assertEquals(readLengths[i], results[i].getReadLength()); - assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock()); - final int[] bOffsets = results[i].getOffsets(); - assertArrayEquals(bufferOffsets[i], bOffsets); - final int[] bLengths = results[i].getLengths(); - assertArrayEquals(bufferLengths[i], bLengths); - } - } - - /** - * Test {@link DFSStripedInputStream#planReadPortions} - */ - @Test - public void testPlanReadPortions() { - /** - * start block offset is 0, read cellSize - 10 - */ - testPlanReadPortions(0, CELLSIZE - 10, 0, - new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0}, - new int[][]{new int[]{0}, new int[]{}, new int[]{}}, - new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}}); - - /** - * start block offset is 0, read 3 * cellSize - */ - testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0, - new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0}, - new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}}, - new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}}); - - /** - * start block offset is 0, read cellSize + 10 - */ - testPlanReadPortions(0, CELLSIZE + 10, 0, - new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0}, - new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}}, - new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}}); - - /** - * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100 - */ - testPlanReadPortions(0, 5 * CELLSIZE + 10, 100, - new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0}, - new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE}, - new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4}, - new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}}, - new int[][]{new int[]{CELLSIZE, CELLSIZE}, - new int[]{CELLSIZE, CELLSIZE}, - new int[]{CELLSIZE, 10}}); - - /** - * start block offset is 2, read 3 * cellSize - */ - testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100, - new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, - new int[]{2, 0, 0}, - new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2}, - new int[]{100 + CELLSIZE - 2}, - new int[]{100 + CELLSIZE * 2 - 2}}, - new int[][]{new int[]{CELLSIZE - 2, 2}, - new int[]{CELLSIZE}, - new int[]{CELLSIZE}}); - - /** - * start block offset is 2, read 3 * cellSize + 10 - */ - testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0, - new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE}, - new int[]{2, 0, 0}, - new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2}, - new int[]{CELLSIZE - 2}, - new int[]{CELLSIZE * 2 - 2}}, - new int[][]{new int[]{CELLSIZE - 2, 12}, - new int[]{CELLSIZE}, - new int[]{CELLSIZE}}); - - /** - * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10 - */ - testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0, - new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2}, - new int[]{CELLSIZE, CELLSIZE - 1, 0}, - new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1}, - new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}, - new int[]{1, 3 * CELLSIZE + 1}}, - new int[][]{new int[]{CELLSIZE, CELLSIZE}, - new int[]{1, CELLSIZE, 9}, - new int[]{CELLSIZE, CELLSIZE}}); - - /** - * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10 - */ - testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0, - new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1}, - new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1}, - new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1}, - new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1}, - new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}}, - new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, - new int[]{CELLSIZE, CELLSIZE, 9}, - new int[]{1, CELLSIZE, CELLSIZE}}); - } - private LocatedStripedBlock createDummyLocatedBlock() { final long blockGroupID = -1048576; DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE]; From 971bd5e885ee1528fd0e7e9486e3b54cde0d02e2 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 10 Apr 2015 04:31:48 +0800 Subject: [PATCH 055/212] HADOOP-11818 Minor improvements for erasurecode classes. Contributed by Rakesh R --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 ++ .../hadoop/io/erasurecode/SchemaLoader.java | 12 ++++++------ .../erasurecode/coder/RSErasureDecoder.java | 19 ++++++++++++++++++- .../erasurecode/coder/RSErasureEncoder.java | 19 ++++++++++++++++++- .../erasurecode/coder/XORErasureDecoder.java | 2 +- .../io/erasurecode/rawcoder/util/RSUtil.java | 17 +++++++++++++++++ 6 files changed, 62 insertions(+), 9 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index c72394e1b1c59..b850e117b8a33 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -40,3 +40,5 @@ HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code ( Kai Zheng via vinayakumarb ) + + HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java index c51ed37608326..75dd03a0652d0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.io.erasurecode; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.w3c.dom.*; @@ -36,7 +36,7 @@ * A EC schema loading utility that loads predefined EC schemas from XML file */ public class SchemaLoader { - private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(SchemaLoader.class.getName()); /** * Load predefined ec schemas from configuration file. This file is @@ -63,7 +63,7 @@ public List loadSchema(Configuration conf) { private List loadSchema(File schemaFile) throws ParserConfigurationException, IOException, SAXException { - LOG.info("Loading predefined EC schema file " + schemaFile); + LOG.info("Loading predefined EC schema file {}", schemaFile); // Read and parse the schema file. DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance(); @@ -87,7 +87,7 @@ private List loadSchema(File schemaFile) ECSchema schema = loadSchema(element); schemas.add(schema); } else { - LOG.warn("Bad element in EC schema configuration file: " + + LOG.warn("Bad element in EC schema configuration file: {}", element.getTagName()); } } @@ -109,7 +109,7 @@ private File getSchemaFile(Configuration conf) { URL url = Thread.currentThread().getContextClassLoader() .getResource(schemaFilePath); if (url == null) { - LOG.warn(schemaFilePath + " not found on the classpath."); + LOG.warn("{} not found on the classpath.", schemaFilePath); schemaFile = null; } else if (! url.getProtocol().equalsIgnoreCase("file")) { throw new RuntimeException( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java index e2c5051ab198c..fc664a502aefd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -1,3 +1,20 @@ +/** + * 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.io.erasurecode.coder; import org.apache.hadoop.conf.Configuration; @@ -11,7 +28,7 @@ /** * Reed-Solomon erasure decoder that decodes a block group. * - * It implements {@link ErasureDecoder}. + * It implements {@link ErasureCoder}. */ public class RSErasureDecoder extends AbstractErasureDecoder { private RawErasureDecoder rsRawDecoder; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java index a7d02b5ab6e1d..18ca5acd910a4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java @@ -1,3 +1,20 @@ +/** + * 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.io.erasurecode.coder; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -9,7 +26,7 @@ /** * Reed-Solomon erasure encoder that encodes a block group. * - * It implements {@link ErasureEncoder}. + * It implements {@link ErasureCoder}. */ public class RSErasureEncoder extends AbstractErasureEncoder { private RawErasureEncoder rawEncoder; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java index 6f4b4238986bb..067254912eee9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java @@ -25,7 +25,7 @@ /** * Xor erasure decoder that decodes a block group. * - * It implements {@link ErasureDecoder}. + * It implements {@link ErasureCoder}. */ public class XORErasureDecoder extends AbstractErasureDecoder { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java index 33ba5614a3abf..8badf021d2f79 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java @@ -1,3 +1,20 @@ +/** + * 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.io.erasurecode.rawcoder.util; /** From 4d0bc724f29b646e252f53d1c654a23e8526a4bf Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 9 Apr 2015 17:53:22 -0700 Subject: [PATCH 056/212] HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. Contributed by Jing Zhao and Zhe Zhang. --- .../BlockInfoStripedUnderConstruction.java | 2 +- .../hdfs/server/blockmanagement/BlockManager.java | 12 ++++++------ .../server/namenode/ErasureCodingZoneManager.java | 7 +++++++ .../hdfs/server/namenode/FSEditLogLoader.java | 11 ++++++----- .../hdfs/server/namenode/FSImageSerialization.java | 4 ++-- .../hadoop/hdfs/server/namenode/INodeFile.java | 14 ++------------ .../hdfs/server/namenode/TestFSEditLogLoader.java | 4 ++-- .../hadoop/hdfs/server/namenode/TestFSImage.java | 2 +- .../server/namenode/TestRecoverStripedBlocks.java | 2 +- 9 files changed, 28 insertions(+), 30 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index cfaf3a0f8caf0..0373314abb987 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -96,7 +96,7 @@ public void setExpectedLocations(DatanodeStorageInfo[] targets) { for(int i = 0; i < numLocations; i++) { // when creating a new block we simply sequentially assign block index to // each storage - Block blk = new Block(this.getBlockId() + i, this.getGenerationStamp(), 0); + Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp()); replicas[i] = new ReplicaUnderConstruction(blk, targets[i], ReplicaState.RBW); } 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 93134bde0f2c0..fd8c68492dcbf 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 @@ -2483,12 +2483,12 @@ private BlockToMarkCorrupt checkReplicaCorrupt( case COMMITTED: if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(reported, storedBlock, reportedGS, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS, "block is " + ucState + " and reported genstamp " + reportedGS + " does not match genstamp in block map " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); } else if (storedBlock.getNumBytes() != reported.getNumBytes()) { - return new BlockToMarkCorrupt(reported, storedBlock, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, "block is " + ucState + " and reported length " + reported.getNumBytes() + " does not match " + "length in block map " + storedBlock.getNumBytes(), @@ -2499,7 +2499,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( case UNDER_CONSTRUCTION: if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(reported, storedBlock, reportedGS, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS, "block is " + ucState + " and reported state " + reportedState + ", But reported genstamp " + reportedGS + " does not match genstamp in block map " @@ -2515,7 +2515,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( return null; // not corrupt } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(reported, storedBlock, reportedGS, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS, "reported " + reportedState + " replica with genstamp " + reportedGS + " does not match COMPLETE block's genstamp in block map " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); @@ -2530,7 +2530,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( "complete with the same genstamp"); return null; } else { - return new BlockToMarkCorrupt(reported, storedBlock, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, "reported replica has invalid state " + reportedState, Reason.INVALID_STATE); } @@ -2543,7 +2543,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( " on " + dn + " size " + storedBlock.getNumBytes(); // log here at WARN level since this is really a broken HDFS invariant LOG.warn(msg); - return new BlockToMarkCorrupt(reported, storedBlock, msg, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, msg, Reason.INVALID_STATE); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index d4ff7c56fe142..606e8048ad54d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -59,6 +59,13 @@ boolean getECPolicy(INodesInPath iip) { if (inode == null) { continue; } + // We don't allow symlinks in an EC zone, or pointing to a file/dir in + // an EC. Therefore if a symlink is encountered, the dir shouldn't have + // EC + // TODO: properly support symlinks in EC zones + if (inode.isSymlink()) { + return false; + } final List xAttrs = inode.getXAttrFeature() == null ? new ArrayList(0) : inode.getXAttrFeature().getXAttrs(); 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 c3ab2d5843bb9..f546b48ef17fc 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 @@ -416,7 +416,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped()); + updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.getECPolicy(iip)); break; } case OP_CLOSE: { @@ -437,7 +437,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped()); + updateBlocks(fsDir, addCloseOp, iip, file, fsDir.getECPolicy(iip)); // Now close the file if (!file.isUnderConstruction() && @@ -496,7 +496,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped()); + updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.getECPolicy(iip)); if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); @@ -510,10 +510,11 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, FSNamesystem.LOG.debug(op.opCode + ": " + path + " new block id : " + addBlockOp.getLastBlock().getBlockId()); } - INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path); + INodesInPath iip = fsDir.getINodesInPath(path, true); + INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // add the new block to the INodeFile // TODO whether the file is striped should later be retrieved from iip - addNewBlock(addBlockOp, oldFile, oldFile.isStriped()); + addNewBlock(addBlockOp, oldFile, fsDir.getECPolicy(iip)); break; } case OP_SET_REPLICATION: { 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 1e58858cdfac8..58244e5bfe104 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 @@ -207,7 +207,7 @@ static void writeINodeUnderConstruction(DataOutputStream out, INodeFile cons, out.writeLong(cons.getModificationTime()); out.writeLong(cons.getPreferredBlockSize()); // whether the file has striped blocks - out.writeBoolean(cons.isWithStripedBlocks()); + out.writeBoolean(cons.isStriped()); writeBlocks(cons.getBlocks(), out); cons.getPermissionStatus().write(out); @@ -233,7 +233,7 @@ public static void writeINodeFile(INodeFile file, DataOutput out, out.writeLong(file.getAccessTime()); out.writeLong(file.getPreferredBlockSize()); // whether the file has striped blocks - out.writeBoolean(file.isWithStripedBlocks()); + out.writeBoolean(file.isStriped()); writeBlocks(file.getBlocks(), out); SnapshotFSImageFormat.saveFileDiffList(file, out); 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 032578a22602d..f87a29c0e2e2a 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 @@ -185,17 +185,13 @@ public final FileWithStripedBlocksFeature getStripedBlocksFeature() { public FileWithStripedBlocksFeature addStripedBlocksFeature() { assert blocks == null || blocks.length == 0: "The file contains contiguous blocks"; - assert !isWithStripedBlocks(); + assert !isStriped(); this.setFileReplication((short) 0); FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature(); addFeature(sb); return sb; } - public boolean isWithStripedBlocks() { - return getStripedBlocksFeature() != null; - } - /** Used to make sure there is no contiguous block related info */ private boolean hasNoContiguousBlock() { return (blocks == null || blocks.length == 0) && getFileReplication() == 0; @@ -432,7 +428,7 @@ private void setFileReplication(short replication) { /** Set the replication factor of this file. */ public final INodeFile setFileReplication(short replication, int latestSnapshotId) throws QuotaExceededException { - Preconditions.checkState(!isWithStripedBlocks(), + Preconditions.checkState(!isStriped(), "Cannot set replication to a file with striped blocks"); recordModification(latestSnapshotId); setFileReplication(replication); @@ -679,7 +675,6 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, final long ssDeltaNoReplication; short replication; - if (isStriped()) { return computeQuotaUsageWithStriped(bsps, counts); } @@ -708,11 +703,6 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, /** * Compute quota of striped file - * @param bsps - * @param counts - * @param useCache - * @param lastSnapshotId - * @return quota counts */ public final QuotaCounts computeQuotaUsageWithStriped( BlockStoragePolicySuite bsps, QuotaCounts counts) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 7dab7eb1280d5..58c53340de43b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -473,7 +473,7 @@ public void testAddNewStripedBlock() throws IOException{ INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory() .getINode(testFilePath); - assertTrue(inodeLoaded.isWithStripedBlocks()); + assertTrue(inodeLoaded.isStriped()); BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks(); assertEquals(1, blks.length); @@ -552,7 +552,7 @@ public void testUpdateStripedBlocks() throws IOException{ INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory() .getINode(testFilePath); - assertTrue(inodeLoaded.isWithStripedBlocks()); + assertTrue(inodeLoaded.isStriped()); BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks(); assertEquals(1, blks.length); 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 7fdf1af05589a..ead65ee4223ed 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 @@ -199,7 +199,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con assertEquals(mtime, fileByLoaded.getModificationTime()); assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime()); assertEquals(0, fileByLoaded.getContiguousBlocks().length); - assertEquals(0, fileByLoaded.getBlockReplication()); + assertEquals(0, fileByLoaded.getFileReplication()); assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize()); //check the BlockInfoStriped diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index b2ff6c881d3e7..4292f9ad6f8e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -84,7 +84,7 @@ public void testMissingStripedBlock() throws Exception { final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() .getINode4Write(filePath.toString()).asFile(); assertFalse(fileNode.isUnderConstruction()); - assertTrue(fileNode.isWithStripedBlocks()); + assertTrue(fileNode.isStriped()); BlockInfo[] blocks = fileNode.getBlocks(); assertEquals(numBlocks, blocks.length); for (BlockInfo blk : blocks) { From 4970f2a2ab8f898e85ddc6e5dd477e844424a999 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Sat, 11 Apr 2015 01:03:37 +0800 Subject: [PATCH 057/212] HDFS-7889 Subclass DFSOutputStream to support writing striping layout files. Contributed by Li Bo --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 4 +- .../apache/hadoop/hdfs/DFSOutputStream.java | 13 +- .../org/apache/hadoop/hdfs/DFSPacket.java | 26 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 439 ++++++++++++++++++ .../org/apache/hadoop/hdfs/DataStreamer.java | 10 +- .../hadoop/hdfs/StripedDataStreamer.java | 241 ++++++++++ .../hdfs/TestDFSStripedOutputStream.java | 311 +++++++++++++ 7 files changed, 1030 insertions(+), 14 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 1e695c430e8f0..753795ad31935 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -56,4 +56,6 @@ HDFS-8074. Define a system-wide default EC schema. (Kai Zheng) - HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng) \ No newline at end of file + HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng) + + HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng) \ No newline at end of file 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 ae5d3eb6a3db9..0280d712a2084 100755 --- 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 @@ -266,8 +266,14 @@ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, } } Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!"); - final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat, - flag, progress, checksum, favoredNodes); + final DFSOutputStream out; + if(stat.getReplication() == 0) { + out = new DFSStripedOutputStream(dfsClient, src, stat, + flag, progress, checksum, favoredNodes); + } else { + out = new DFSOutputStream(dfsClient, src, stat, + flag, progress, checksum, favoredNodes); + } out.start(); return out; } finally { @@ -347,6 +353,9 @@ static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src, String[] favoredNodes) throws IOException { TraceScope scope = dfsClient.getPathTraceScope("newStreamForAppend", src); + if(stat.getReplication() == 0) { + throw new IOException("Not support appending to a striping layout file yet."); + } try { final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags, progress, lastBlock, stat, checksum, favoredNodes); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java index 22055c319007d..9cd1ec1fcc33e 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java @@ -20,6 +20,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.BufferOverflowException; +import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.util.Arrays; @@ -113,6 +114,19 @@ synchronized void writeData(byte[] inarray, int off, int len) dataPos += len; } + synchronized void writeData(ByteBuffer inBuffer, int len) + throws ClosedChannelException { + checkBuffer(); + len = len > inBuffer.remaining() ? inBuffer.remaining() : len; + if (dataPos + len > buf.length) { + throw new BufferOverflowException(); + } + for (int i = 0; i < len; i++) { + buf[dataPos + i] = inBuffer.get(); + } + dataPos += len; + } + /** * Write checksums to this packet * @@ -222,7 +236,7 @@ boolean isHeartbeatPacket() { * * @return true if the packet is the last packet */ - boolean isLastPacketInBlock(){ + boolean isLastPacketInBlock() { return lastPacketInBlock; } @@ -231,7 +245,7 @@ boolean isLastPacketInBlock(){ * * @return the sequence number of this packet */ - long getSeqno(){ + long getSeqno() { return seqno; } @@ -240,14 +254,14 @@ long getSeqno(){ * * @return the number of chunks in this packet */ - synchronized int getNumChunks(){ + synchronized int getNumChunks() { return numChunks; } /** * increase the number of chunks by one */ - synchronized void incNumChunks(){ + synchronized void incNumChunks() { numChunks++; } @@ -256,7 +270,7 @@ synchronized void incNumChunks(){ * * @return the maximum number of packets */ - int getMaxChunks(){ + int getMaxChunks() { return maxChunks; } @@ -265,7 +279,7 @@ int getMaxChunks(){ * * @param syncBlock if to sync block */ - synchronized void setSyncBlock(boolean syncBlock){ + synchronized void setSyncBlock(boolean syncBlock) { this.syncBlock = syncBlock; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java new file mode 100644 index 0000000000000..aded4fe6ef741 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -0,0 +1,439 @@ +/** + * 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 java.io.IOException; +import java.io.InterruptedIOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; +import org.apache.hadoop.util.DataChecksum; +import org.apache.hadoop.util.Progressable; +import org.apache.htrace.Sampler; +import org.apache.htrace.Trace; +import org.apache.htrace.TraceScope; + + +/**************************************************************** + * The DFSStripedOutputStream class supports writing files in striped + * layout. Each stripe contains a sequence of cells and multiple + * {@link StripedDataStreamer}s in DFSStripedOutputStream are responsible + * for writing the cells to different datanodes. + * + ****************************************************************/ + +@InterfaceAudience.Private +public class DFSStripedOutputStream extends DFSOutputStream { + + private final List streamers; + /** + * Size of each striping cell, must be a multiple of bytesPerChecksum + */ + private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private ByteBuffer[] cellBuffers; + private final short blockGroupBlocks = HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; + private final short blockGroupDataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private int curIdx = 0; + /* bytes written in current block group */ + private long currentBlockGroupBytes = 0; + + //TODO: Use ErasureCoder interface (HDFS-7781) + private RawErasureEncoder encoder; + + private StripedDataStreamer getLeadingStreamer() { + return streamers.get(0); + } + + private long getBlockGroupSize() { + return blockSize * HdfsConstants.NUM_DATA_BLOCKS; + } + + /** Construct a new output stream for creating a file. */ + DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, + EnumSet flag, Progressable progress, + DataChecksum checksum, String[] favoredNodes) + throws IOException { + super(dfsClient, src, stat, flag, progress, checksum, favoredNodes); + DFSClient.LOG.info("Creating striped output stream"); + if (blockGroupBlocks <= 1) { + throw new IOException("The block group must contain more than one block."); + } + + cellBuffers = new ByteBuffer[blockGroupBlocks]; + List> stripeBlocks = new ArrayList<>(); + + for (int i = 0; i < blockGroupBlocks; i++) { + stripeBlocks.add(new LinkedBlockingQueue(blockGroupBlocks)); + try { + cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize)); + } catch (InterruptedException ie) { + final InterruptedIOException iioe = new InterruptedIOException( + "create cell buffers"); + iioe.initCause(ie); + throw iioe; + } + } + encoder = new RSRawEncoder(); + encoder.initialize(blockGroupDataBlocks, + blockGroupBlocks - blockGroupDataBlocks, cellSize); + + streamers = new ArrayList<>(blockGroupBlocks); + for (short i = 0; i < blockGroupBlocks; i++) { + StripedDataStreamer streamer = new StripedDataStreamer(stat, null, + dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager, + i, stripeBlocks); + if (favoredNodes != null && favoredNodes.length != 0) { + streamer.setFavoredNodes(favoredNodes); + } + streamers.add(streamer); + } + + refreshStreamer(); + } + + private void refreshStreamer() { + streamer = streamers.get(curIdx); + } + + private void moveToNextStreamer() { + curIdx = (curIdx + 1) % blockGroupBlocks; + refreshStreamer(); + } + + /** + * encode the buffers. + * After encoding, flip each buffer. + * + * @param buffers data buffers + parity buffers + */ + private void encode(ByteBuffer[] buffers) { + ByteBuffer[] dataBuffers = new ByteBuffer[blockGroupDataBlocks]; + ByteBuffer[] parityBuffers = new ByteBuffer[blockGroupBlocks - blockGroupDataBlocks]; + for (int i = 0; i < blockGroupBlocks; i++) { + if (i < blockGroupDataBlocks) { + dataBuffers[i] = buffers[i]; + } else { + parityBuffers[i - blockGroupDataBlocks] = buffers[i]; + } + } + encoder.encode(dataBuffers, parityBuffers); + } + + /** + * Generate packets from a given buffer + * + * @param byteBuffer the given buffer to generate packets + * @return packets generated + * @throws IOException + */ + private List generatePackets(ByteBuffer byteBuffer) + throws IOException{ + List packets = new ArrayList<>(); + while (byteBuffer.remaining() > 0) { + DFSPacket p = createPacket(packetSize, chunksPerPacket, + streamer.getBytesCurBlock(), + streamer.getAndIncCurrentSeqno(), false); + int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum; + int toWrite = byteBuffer.remaining() > maxBytesToPacket ? + maxBytesToPacket: byteBuffer.remaining(); + p.writeData(byteBuffer, toWrite); + streamer.incBytesCurBlock(toWrite); + packets.add(p); + } + return packets; + } + + @Override + protected synchronized void writeChunk(byte[] b, int offset, int len, + byte[] checksum, int ckoff, int cklen) throws IOException { + super.writeChunk(b, offset, len, checksum, ckoff, cklen); + + if (getSizeOfCellnBuffer(curIdx) <= cellSize) { + addToCellBuffer(b, offset, len); + } else { + String msg = "Writing a chunk should not overflow the cell buffer."; + DFSClient.LOG.info(msg); + throw new IOException(msg); + } + + + // If current packet has not been enqueued for transmission, + // but the cell buffer is full, we need to enqueue the packet + if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) { + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("DFSClient writeChunk cell buffer full seqno=" + + currentPacket.getSeqno() + + ", curIdx=" + curIdx + + ", src=" + src + + ", bytesCurBlock=" + streamer.getBytesCurBlock() + + ", blockSize=" + blockSize + + ", appendChunk=" + streamer.getAppendChunk()); + } + streamer.waitAndQueuePacket(currentPacket); + currentPacket = null; + adjustChunkBoundary(); + endBlock(); + } + + // Two extra steps are needed when a striping cell is full: + // 1. Forward the current index pointer + // 2. Generate parity packets if a full stripe of data cells are present + if (getSizeOfCellnBuffer(curIdx) == cellSize) { + //move curIdx to next cell + moveToNextStreamer(); + //When all data cells in a stripe are ready, we need to encode + //them and generate some parity cells. These cells will be + //converted to packets and put to their DataStreamer's queue. + if (curIdx == blockGroupDataBlocks) { + //encode the data cells + for (int k = 0; k < blockGroupDataBlocks; k++) { + cellBuffers[k].flip(); + } + encode(cellBuffers); + for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) { + ByteBuffer parityBuffer = cellBuffers[i]; + List packets = generatePackets(parityBuffer); + for (DFSPacket p : packets) { + currentPacket = p; + streamer.waitAndQueuePacket(currentPacket); + currentPacket = null; + } + endBlock(); + moveToNextStreamer(); + } + //read next stripe to cellBuffers + clearCellBuffers(); + } + } + } + + private void addToCellBuffer(byte[] b, int off, int len) { + cellBuffers[curIdx].put(b, off, len); + } + + private int getSizeOfCellnBuffer(int cellIndex) { + return cellBuffers[cellIndex].position(); + } + + private void clearCellBuffers() { + for (int i = 0; i< blockGroupBlocks; i++) { + cellBuffers[i].clear(); + } + } + + private int stripeDataSize() { + return blockGroupDataBlocks * cellSize; + } + + private void notSupported(String headMsg) + throws IOException{ + throw new IOException( + headMsg + " is now not supported for striping layout."); + } + + @Override + public void hflush() throws IOException { + notSupported("hflush"); + } + + @Override + public void hsync() throws IOException { + notSupported("hsync"); + } + + + @Override + protected synchronized void start() { + for (StripedDataStreamer streamer : streamers) { + streamer.start(); + } + } + + @Override + synchronized void abort() throws IOException { + if (isClosed()) { + return; + } + for (StripedDataStreamer streamer : streamers) { + streamer.setLastException(new IOException("Lease timeout of " + + (dfsClient.getHdfsTimeout()/1000) + " seconds expired.")); + } + closeThreads(true); + dfsClient.endFileLease(fileId); + } + + //TODO: Handle slow writers (HDFS-7786) + //Cuurently only check if the leading streamer is terminated + boolean isClosed() { + return closed || getLeadingStreamer().streamerClosed(); + } + + // shutdown datastreamer and responseprocessor threads. + // interrupt datastreamer if force is true + @Override + protected void closeThreads(boolean force) throws IOException { + StripedDataStreamer leadingStreamer = null; + for (StripedDataStreamer streamer : streamers) { + try { + streamer.close(force); + streamer.join(); + streamer.closeSocket(); + if (streamer.isLeadingStreamer()) { + leadingStreamer = streamer; + } else { + streamer.countTailingBlockGroupBytes(); + } + + } catch (InterruptedException e) { + throw new IOException("Failed to shutdown streamer"); + } finally { + streamer.setSocketToNull(); + setClosed(); + } + } + leadingStreamer.countTailingBlockGroupBytes(); + } + + @Override + public synchronized void write(int b) throws IOException { + super.write(b); + currentBlockGroupBytes = (currentBlockGroupBytes + 1) % getBlockGroupSize(); + } + + @Override + public synchronized void write(byte b[], int off, int len) + throws IOException { + super.write(b, off, len); + currentBlockGroupBytes = (currentBlockGroupBytes + len) % getBlockGroupSize(); + } + + private void writeParityCellsForLastStripe() throws IOException{ + if(currentBlockGroupBytes == 0 || + currentBlockGroupBytes % stripeDataSize() == 0) + return; + int lastStripeLen =(int)(currentBlockGroupBytes % stripeDataSize()); + // Size of parity cells should equal the size of the first cell, if it + // is not full. + int parityCellSize = cellSize; + int index = lastStripeLen / cellSize; + if (lastStripeLen < cellSize) { + parityCellSize = lastStripeLen; + index++; + } + for (int i = 0; i < blockGroupBlocks; i++) { + if (i >= index) { + int position = cellBuffers[i].position(); + for (int j = 0; j < parityCellSize - position; j++) { + cellBuffers[i].put((byte)0); + } + } + cellBuffers[i].flip(); + } + encode(cellBuffers); + + //write parity cells + curIdx = blockGroupDataBlocks; + refreshStreamer(); + for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) { + ByteBuffer parityBuffer = cellBuffers[i]; + List packets = generatePackets(parityBuffer); + for (DFSPacket p : packets) { + currentPacket = p; + streamer.waitAndQueuePacket(currentPacket); + currentPacket = null; + } + endBlock(); + moveToNextStreamer(); + } + + clearCellBuffers(); + } + + @Override + void setClosed() { + super.setClosed(); + for (int i = 0; i < blockGroupBlocks; i++) { + byteArrayManager.release(cellBuffers[i].array()); + streamers.get(i).release(); + } + } + + @Override + protected synchronized void closeImpl() throws IOException { + if (isClosed()) { + IOException e = getLeadingStreamer().getLastException().getAndSet(null); + if (e == null) + return; + else + throw e; + } + + try { + // flush from all upper layers + flushBuffer(); + if (currentPacket != null) { + streamer.waitAndQueuePacket(currentPacket); + currentPacket = null; + } + //if the last stripe is incomplete, generate and write parity cells + writeParityCellsForLastStripe(); + + for (int i = 0; i < blockGroupBlocks; i++) { + curIdx = i; + refreshStreamer(); + if (streamer.getBytesCurBlock()!= 0 || + currentBlockGroupBytes < getBlockGroupSize()) { + // send an empty packet to mark the end of the block + currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), + streamer.getAndIncCurrentSeqno(), true); + currentPacket.setSyncBlock(shouldSyncBlock); + } + // flush all data to Datanode + flushInternal(); + } + + // get last block before destroying the streamer + ExtendedBlock lastBlock = streamers.get(0).getBlock(); + closeThreads(false); + TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER); + try { + completeFile(lastBlock); + } finally { + scope.close(); + } + dfsClient.endFileLease(fileId); + } catch (ClosedChannelException e) { + } finally { + setClosed(); + } + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index cecd5a0eccf4c..631f3869a299d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -336,7 +336,7 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { } private volatile boolean streamerClosed = false; - private ExtendedBlock block; // its length is number of bytes acked + protected ExtendedBlock block; // its length is number of bytes acked private Token accessToken; private DataOutputStream blockStream; private DataInputStream blockReplyStream; @@ -367,12 +367,12 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { private final LastExceptionInStreamer lastException = new LastExceptionInStreamer(); private Socket s; - private final DFSClient dfsClient; - private final String src; + protected final DFSClient dfsClient; + protected final String src; /** Only for DataTransferProtocol.writeBlock(..) */ private final DataChecksum checksum4WriteBlock; private final Progressable progress; - private final HdfsFileStatus stat; + protected final HdfsFileStatus stat; // appending to existing partial block private volatile boolean appendChunk = false; // both dataQueue and ackQueue are protected by dataQueue lock @@ -489,7 +489,7 @@ private void initDataStreaming() { stage = BlockConstructionStage.DATA_STREAMING; } - private void endBlock() { + protected void endBlock() { if(LOG.isDebugEnabled()) { LOG.debug("Closing old block " + block); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java new file mode 100644 index 0000000000000..710d92d7363c3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -0,0 +1,241 @@ +/** + * 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 java.util.List; +import org.apache.hadoop.fs.StorageType; +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.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.util.ByteArrayManager; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.DataChecksum; +import org.apache.hadoop.util.Progressable; + +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +/**************************************************************************** + * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}. + * There are two kinds of StripedDataStreamer, leading streamer and ordinary + * stream. Leading streamer requests a block group from NameNode, unwraps + * it to located blocks and transfers each located block to its corresponding + * ordinary streamer via a blocking queue. + * + ****************************************************************************/ +public class StripedDataStreamer extends DataStreamer { + private final short index; + private final List> stripedBlocks; + private static short blockGroupSize = HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; + private boolean hasCommittedBlock = false; + + StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block, + DFSClient dfsClient, String src, + Progressable progress, DataChecksum checksum, + AtomicReference cachingStrategy, + ByteArrayManager byteArrayManage, short index, + List> stripedBlocks) { + super(stat,block, dfsClient, src, progress, checksum, cachingStrategy, + byteArrayManage); + this.index = index; + this.stripedBlocks = stripedBlocks; + } + + /** + * Construct a data streamer for appending to the last partial block + * @param lastBlock last block of the file to be appended + * @param stat status of the file to be appended + * @throws IOException if error occurs + */ + StripedDataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, + DFSClient dfsClient, String src, + Progressable progress, DataChecksum checksum, + AtomicReference cachingStrategy, + ByteArrayManager byteArrayManage, short index, + List> stripedBlocks) + throws IOException { + super(lastBlock, stat, dfsClient, src, progress, checksum, cachingStrategy, + byteArrayManage); + this.index = index; + this.stripedBlocks = stripedBlocks; + } + + public boolean isLeadingStreamer () { + return index == 0; + } + + private boolean isParityStreamer() { + return index >= HdfsConstants.NUM_DATA_BLOCKS; + } + + @Override + protected void endBlock() { + if (!isLeadingStreamer() && !isParityStreamer()) { + //before retrieving a new block, transfer the finished block to + //leading streamer + LocatedBlock finishedBlock = new LocatedBlock( + new ExtendedBlock(block.getBlockPoolId(), block.getBlockId(), + block.getNumBytes(),block.getGenerationStamp()), null); + try{ + boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30, + TimeUnit.SECONDS); + }catch (InterruptedException ie) { + //TODO: Handle InterruptedException (HDFS-7786) + } + } + super.endBlock(); + } + + /** + * This function is called after the streamer is closed. + */ + void countTailingBlockGroupBytes () throws IOException { + if (isLeadingStreamer()) { + //when committing a block group, leading streamer has to adjust + // {@link block} including the size of block group + for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) { + try { + LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30, + TimeUnit.SECONDS); + if (finishedLocatedBlock == null) { + throw new IOException("Fail to get finished LocatedBlock " + + "from streamer, i=" + i); + } + ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock(); + long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes(); + if (block != null) { + block.setNumBytes(block.getNumBytes() + bytes); + } + } catch (InterruptedException ie) { + DFSClient.LOG.info("InterruptedException received when " + + "putting a block to stripeBlocks, ie = " + ie); + } + } + } else if (!isParityStreamer()) { + if (block == null || block.getNumBytes() == 0) { + LocatedBlock finishedBlock = new LocatedBlock(null, null); + try { + boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30, + TimeUnit.SECONDS); + } catch (InterruptedException ie) { + //TODO: Handle InterruptedException (HDFS-7786) + ie.printStackTrace(); + } + } + } + + } + + @Override + protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) + throws IOException { + LocatedBlock lb = null; + if (isLeadingStreamer()) { + if(hasCommittedBlock) { + //when committing a block group, leading streamer has to adjust + // {@link block} including the size of block group + for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) { + try { + LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30, + TimeUnit.SECONDS); + if (finishedLocatedBlock == null) { + throw new IOException("Fail to get finished LocatedBlock " + + "from streamer, i=" + i); + } + ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock(); + long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes(); + if(block != null) { + block.setNumBytes(block.getNumBytes() + bytes); + } + } catch (InterruptedException ie) { + DFSClient.LOG.info("InterruptedException received when putting" + + " a block to stripeBlocks, ie = " + ie); + } + } + } + + lb = super.locateFollowingBlock(excludedNodes); + hasCommittedBlock = true; + LocatedBlock[] blocks = unwrapBlockGroup(lb); + assert blocks.length == blockGroupSize : + "Fail to get block group from namenode: blockGroupSize: " + + blockGroupSize + ", blocks.length: " + blocks.length; + lb = blocks[0]; + for (int i = 1; i < blocks.length; i++) { + try { + boolean offSuccess = stripedBlocks.get(i).offer(blocks[i], + 90, TimeUnit.SECONDS); + if(!offSuccess){ + String msg = "Fail to put block to stripeBlocks. i = " + i; + DFSClient.LOG.info(msg); + throw new IOException(msg); + } else { + DFSClient.LOG.info("Allocate a new block to a streamer. i = " + i + + ", block: " + blocks[i]); + } + } catch (InterruptedException ie) { + DFSClient.LOG.info("InterruptedException received when putting" + + " a block to stripeBlocks, ie = " + ie); + } + } + } else { + try { + //wait 90 seconds to get a block from the queue + lb = stripedBlocks.get(index).poll(90, TimeUnit.SECONDS); + } catch (InterruptedException ie) { + DFSClient.LOG.info("InterruptedException received when retrieving " + + "a block from stripeBlocks, ie = " + ie); + } + } + return lb; + } + + /** + * Generate other blocks in a block group according to the first one. + * + * @param firstBlockInGroup the first block in a block group + * @return other blocks in this group + */ + public static LocatedBlock[] unwrapBlockGroup( + final LocatedBlock firstBlockInGroup) { + ExtendedBlock eb = firstBlockInGroup.getBlock(); + DatanodeInfo[] locs = firstBlockInGroup.getLocations(); + String[] storageIDs = firstBlockInGroup.getStorageIDs(); + StorageType[] storageTypes = firstBlockInGroup.getStorageTypes(); + Token blockToken = firstBlockInGroup.getBlockToken(); + LocatedBlock[] blocksInGroup = new LocatedBlock[locs.length]; + for (int i = 0; i < blocksInGroup.length; i++) { + //each block in a group has the same number of bytes and timestamp + ExtendedBlock extendedBlock = new ExtendedBlock(eb.getBlockPoolId(), + eb.getBlockId() + i, eb.getNumBytes(), eb.getGenerationStamp()); + blocksInGroup[i] = new LocatedBlock(extendedBlock, + new DatanodeInfo[] {locs[i]}, new String[]{storageIDs[i]}, + new StorageType[] {storageTypes[i]}); + blocksInGroup[i].setBlockToken(blockToken); + } + return blocksInGroup; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java new file mode 100644 index 0000000000000..f5a37f353da50 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -0,0 +1,311 @@ +package org.apache.hadoop.hdfs; + +import java.util.Arrays; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.net.Peer; +import org.apache.hadoop.hdfs.net.TcpPeerServer; +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.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; + +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.token.Token; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.ArrayList; +import java.util.List; + +public class TestDFSStripedOutputStream { + private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + + private MiniDFSCluster cluster; + private Configuration conf = new Configuration(); + private DistributedFileSystem fs; + int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + int blockSize = 8 * 1024 * 1024; + int cellsInBlock = blockSize / cellSize; + private int mod = 29; + + @Before + public void setup() throws IOException { + int numDNs = dataBlocks + parityBlocks + 2; + Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().createErasureCodingZone("/"); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void TestFileEmpty() throws IOException { + testOneFile("/EmptyFile", 0); + } + + @Test + public void TestFileSmallerThanOneCell1() throws IOException { + testOneFile("/SmallerThanOneCell", 1); + } + + @Test + public void TestFileSmallerThanOneCell2() throws IOException { + testOneFile("/SmallerThanOneCell", cellSize - 1); + } + + @Test + public void TestFileEqualsWithOneCell() throws IOException { + testOneFile("/EqualsWithOneCell", cellSize); + } + + @Test + public void TestFileSmallerThanOneStripe1() throws IOException { + testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1); + } + + @Test + public void TestFileSmallerThanOneStripe2() throws IOException { + testOneFile("/SmallerThanOneStripe", cellSize + 123); + } + + @Test + public void TestFileEqualsWithOneStripe() throws IOException { + testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks); + } + + @Test + public void TestFileMoreThanOneStripe1() throws IOException { + testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123); + } + + @Test + public void TestFileMoreThanOneStripe2() throws IOException { + testOneFile("/MoreThanOneStripe2", + cellSize * dataBlocks * (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) + + cellSize * dataBlocks + 123); + } + + @Test + public void TestFileFullBlockGroup() throws IOException { + testOneFile("/FullBlockGroup", blockSize * dataBlocks); + } + + //TODO: The following tests will pass after HDFS-8121 fixed +// @Test + public void TestFileMoreThanABlockGroup1() throws IOException { + testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); + } + + // @Test + public void TestFileMoreThanABlockGroup2() throws IOException { + testOneFile("/MoreThanABlockGroup2", + blockSize * dataBlocks * 3 + + (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) * cellSize * dataBlocks + + 123); + } + + private int stripeDataSize() { + return cellSize * dataBlocks; + } + + private byte[] generateBytes(int cnt) { + byte[] bytes = new byte[cnt]; + for (int i = 0; i < cnt; i++) { + bytes[i] = getByte(i); + } + return bytes; + } + + private byte getByte(long pos) { + return (byte) (pos % mod + 1); + } + + private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) + throws IOException { + Path TestPath = new Path(src); + byte[] bytes = generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); + + //check file length + FileStatus status = fs.getFileStatus(TestPath); + long fileLength = status.getLen(); + if (fileLength != writeBytes) { + Assert.fail("File Length error: expect=" + writeBytes + + ", actual=" + fileLength); + } + + DFSStripedInputStream dis = new DFSStripedInputStream( + fs.getClient(), src, true); + byte[] buf = new byte[writeBytes + 100]; + int readLen = dis.read(0, buf, 0, buf.length); + readLen = readLen >= 0 ? readLen : 0; + if (readLen != writeBytes) { + Assert.fail("The length of file is not correct."); + } + + for (int i = 0; i < writeBytes; i++) { + if (getByte(i) != buf[i]) { + Assert.fail("Byte at i = " + i + " is wrongly written."); + } + } + } + + private void testOneFile(String src, int writeBytes) + throws IOException { + Path TestPath = new Path(src); + + int allBlocks = dataBlocks + parityBlocks; + byte[] bytes = generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); + + //check file length + FileStatus status = fs.getFileStatus(TestPath); + long fileLength = status.getLen(); + if (fileLength != writeBytes) { + Assert.fail("File Length error: expect=" + writeBytes + + ", actual=" + fileLength); + } + + List> blockGroupList = new ArrayList<>(); + LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L); + + for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { + LocatedBlock[] blocks = StripedDataStreamer.unwrapBlockGroup(firstBlock); + List oneGroup = Arrays.asList(blocks); + blockGroupList.add(oneGroup); + } + + //test each block group + for (int group = 0; group < blockGroupList.size(); group++) { + //get the data of this block + List blockList = blockGroupList.get(group); + byte[][] dataBlockBytes = new byte[dataBlocks][]; + byte[][] parityBlockBytes = new byte[allBlocks - dataBlocks][]; + + //calculate the size of this block group + int lenOfBlockGroup = group < blockGroupList.size() - 1 ? + blockSize * dataBlocks : + writeBytes - blockSize * (blockGroupList.size() - 1) * dataBlocks; + int intactStripes = lenOfBlockGroup / stripeDataSize(); + int lastStripeLen = lenOfBlockGroup % stripeDataSize(); + + //for each block, use BlockReader to read data + for (int i = 0; i < blockList.size(); i++) { + LocatedBlock lblock = blockList.get(i); + if (lblock == null) { + continue; + } + DatanodeInfo[] nodes = lblock.getLocations(); + ExtendedBlock block = lblock.getBlock(); + InetSocketAddress targetAddr = NetUtils.createSocketAddr( + nodes[0].getXferAddr()); + + int lenOfCell = cellSize; + if (i == lastStripeLen / cellSize) { + lenOfCell = lastStripeLen % cellSize; + } else if (i > lastStripeLen / cellSize) { + lenOfCell = 0; + } + int lenOfBlock = cellSize * intactStripes + lenOfCell; + byte[] blockBytes = new byte[lenOfBlock]; + if (i < dataBlocks) { + dataBlockBytes[i] = blockBytes; + } else { + parityBlockBytes[i - dataBlocks] = blockBytes; + } + + if (lenOfBlock == 0) { + continue; + } + + block.setNumBytes(lenOfBlock); + BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)). + setFileName(src). + setBlock(block). + setBlockToken(lblock.getBlockToken()). + setInetSocketAddress(targetAddr). + setStartOffset(0). + setLength(block.getNumBytes()). + setVerifyChecksum(true). + setClientName("TestStripeLayoutWrite"). + setDatanodeInfo(nodes[0]). + setCachingStrategy(CachingStrategy.newDefaultStrategy()). + setClientCacheContext(ClientContext.getFromConf(conf)). + setConfiguration(conf). + setRemotePeerFactory(new RemotePeerFactory() { + @Override + public Peer newConnectedPeer(InetSocketAddress addr, + Token blockToken, + DatanodeID datanodeId) + throws IOException { + Peer peer = null; + Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); + try { + sock.connect(addr, HdfsServerConstants.READ_TIMEOUT); + sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); + peer = TcpPeerServer.peerFromSocket(sock); + } finally { + if (peer == null) { + IOUtils.closeSocket(sock); + } + } + return peer; + } + }).build(); + + blockReader.readAll(blockBytes, 0, lenOfBlock); + blockReader.close(); + } + + //check if we write the data correctly + for (int i = 0; i < dataBlockBytes.length; i++) { + byte[] cells = dataBlockBytes[i]; + if (cells == null) { + continue; + } + for (int j = 0; j < cells.length; j++) { + byte expected; + //calculate the postion of this byte in the file + long pos = group * dataBlocks * blockSize + + (i * cellSize + j / cellSize * cellSize * dataBlocks) + + j % cellSize; + if (pos >= writeBytes) { + expected = 0; + } else { + expected = getByte(pos); + } + + if (expected != cells[j]) { + Assert.fail("Unexpected byte " + cells[j] + ", expect " + expected + + ". Block group index is " + group + + ", stripe index is " + j / cellSize + + ", cell index is " + i + ", byte index is " + j % cellSize); + } + } + } + } + } + +} From f53e402635a990458113ba0595655751d83c02e1 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Fri, 10 Apr 2015 15:07:32 +0530 Subject: [PATCH 058/212] HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. (Contributed by Vinayakumar B) --- .../hadoop/io/erasurecode/ECSchema.java | 4 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +- .../org/apache/hadoop/hdfs/DFSClient.java | 11 ++++ .../hadoop/hdfs/protocol/ClientProtocol.java | 10 ++++ ...amenodeProtocolServerSideTranslatorPB.java | 19 +++++++ .../ClientNamenodeProtocolTranslatorPB.java | 26 ++++++++- .../hadoop/hdfs/protocolPB/PBHelper.java | 5 +- .../hdfs/server/namenode/FSNamesystem.java | 17 ++++++ .../server/namenode/NameNodeRpcServer.java | 9 ++- .../main/proto/ClientNamenodeProtocol.proto | 9 +++ .../hadoop-hdfs/src/main/proto/hdfs.proto | 3 +- .../org/apache/hadoop/hdfs/TestECSchemas.java | 57 +++++++++++++++++++ 12 files changed, 164 insertions(+), 11 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index 8c3310e406717..32077f6630667 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -123,12 +123,12 @@ private void initWith(String codecName, int numDataUnits, int numParityUnits, this.chunkSize = DEFAULT_CHUNK_SIZE; try { - if (options.containsKey(CHUNK_SIZE_KEY)) { + if (this.options.containsKey(CHUNK_SIZE_KEY)) { this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY)); } } catch (NumberFormatException e) { throw new IllegalArgumentException("Option value " + - options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + this.options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + " is found. It should be an integer"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 753795ad31935..5250dfa271cec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -58,4 +58,7 @@ HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng) - HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng) \ No newline at end of file + HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng) + + HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all + ECSchemas loaded in Namenode. (vinayakumarb) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index c418c4d57fef3..2acbf87d38647 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -165,6 +165,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.retry.LossyRetryInvocationHandler; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RemoteException; @@ -3149,6 +3150,16 @@ public ECInfo getErasureCodingInfo(String src) throws IOException { } } + public ECSchema[] getECSchemas() throws IOException { + checkOpen(); + TraceScope scope = Trace.startSpan("getECSchemas", traceSampler); + try { + return namenode.getECSchemas(); + } finally { + scope.close(); + } + } + public DFSInotifyEventInputStream getInotifyEventStream() throws IOException { checkOpen(); return new DFSInotifyEventInputStream(traceSampler, namenode); 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 d27dac7c3d33e..3fc02014eb1fe 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 @@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.retry.AtMostOnce; import org.apache.hadoop.io.retry.Idempotent; import org.apache.hadoop.security.AccessControlException; @@ -1474,4 +1475,13 @@ public List listXAttrs(String src) */ @Idempotent public ECInfo getErasureCodingInfo(String src) throws IOException; + + /** + * Gets list of ECSchemas loaded in Namenode + * + * @return Returns the list of ECSchemas loaded at Namenode + * @throws IOException + */ + @Idempotent + public ECSchema[] getECSchemas() throws IOException; } 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 4b110fe3a1851..e8f513ac1fa09 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 @@ -107,6 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto; @@ -218,6 +220,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto; import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; @@ -1530,4 +1533,20 @@ public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController cont throw new ServiceException(e); } } + + @Override + public GetECSchemasResponseProto getECSchemas(RpcController controller, + GetECSchemasRequestProto request) throws ServiceException { + try { + ECSchema[] ecSchemas = server.getECSchemas(); + GetECSchemasResponseProto.Builder resBuilder = GetECSchemasResponseProto + .newBuilder(); + for (ECSchema ecSchema : ecSchemas) { + resBuilder.addSchemas(PBHelper.convertECSchema(ecSchema)); + } + return resBuilder.build(); + } 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 568da687436ef..02115227e5ae0 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 @@ -107,6 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto; @@ -165,10 +167,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos; +import org.apache.hadoop.hdfs.protocol.proto.*; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; @@ -180,6 +183,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolMetaInterface; import org.apache.hadoop.ipc.ProtocolTranslator; @@ -237,6 +241,10 @@ public class ClientNamenodeProtocolTranslatorPB implements VOID_GET_STORAGE_POLICIES_REQUEST = GetStoragePoliciesRequestProto.newBuilder().build(); + private final static GetECSchemasRequestProto + VOID_GET_ECSCHEMAS_REQUEST = GetECSchemasRequestProto + .newBuilder().build(); + public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) { rpcProxy = proxy; } @@ -1550,4 +1558,20 @@ public ECInfo getErasureCodingInfo(String src) throws IOException { throw ProtobufHelper.getRemoteException(e); } } + + @Override + public ECSchema[] getECSchemas() throws IOException { + try { + GetECSchemasResponseProto response = rpcProxy.getECSchemas(null, + VOID_GET_ECSCHEMAS_REQUEST); + ECSchema[] schemas = new ECSchema[response.getSchemasCount()]; + int i = 0; + for (ECSchemaProto schemaProto : response.getSchemasList()) { + schemas[i++] = PBHelper.convertECSchema(schemaProto); + } + return schemas; + } 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 c0a482cd4e29e..c169515c8276a 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 @@ -3121,8 +3121,6 @@ public static ECSchema convertECSchema(ECSchemaProto schema) { for (ECSchemaOptionEntryProto option : optionsList) { options.put(option.getKey(), option.getValue()); } - // include chunksize in options. - options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(schema.getChunkSize())); return new ECSchema(schema.getSchemaName(), schema.getCodecName(), schema.getDataUnits(), schema.getParityUnits(), options); } @@ -3132,8 +3130,7 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { .setSchemaName(schema.getSchemaName()) .setCodecName(schema.getCodecName()) .setDataUnits(schema.getNumDataUnits()) - .setParityUnits(schema.getNumParityUnits()) - .setChunkSize(schema.getChunkSize()); + .setParityUnits(schema.getNumParityUnits()); Set> entrySet = schema.getOptions().entrySet(); for (Entry entry : entrySet) { builder.addOptions(ECSchemaOptionEntryProto.newBuilder() 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 c2bca5e199de2..33cbdd1453229 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 @@ -7585,6 +7585,23 @@ ECInfo getErasureCodingInfo(String src) throws AccessControlException, return null; } + /** + * Get available ECSchemas + */ + ECSchema[] getECSchemas() throws IOException { + checkOperation(OperationCategory.READ); + waitForLoadingFSImage(); + readLock(); + try { + checkOperation(OperationCategory.READ); + // TODO HDFS-7866 Need to return all schemas maintained by Namenode + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + return new ECSchema[] { defaultSchema }; + } finally { + readUnlock(); + } + } + void setXAttr(String src, XAttr xAttr, EnumSet flag, boolean logRetryCache) throws IOException { 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 2d19e47f5acce..d0bc453bbe594 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 @@ -143,6 +143,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RetryCache; @@ -2031,9 +2032,15 @@ public void removeSpanReceiver(long id) throws IOException { nn.spanReceiverHost.removeSpanReceiver(id); } - @Override // ClientNameNodeProtocol + @Override // ClientProtocol public ECInfo getErasureCodingInfo(String src) throws IOException { checkNNStartup(); return namesystem.getErasureCodingInfo(src); } + + @Override // ClientProtocol + public ECSchema[] getECSchemas() throws IOException { + checkNNStartup(); + return namesystem.getECSchemas(); + } } 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 9488aed8c17ee..3389a22350531 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -729,6 +729,13 @@ message GetErasureCodingInfoResponseProto { optional ECInfoProto ECInfo = 1; } +message GetECSchemasRequestProto { // void request +} + +message GetECSchemasResponseProto { + repeated ECSchemaProto schemas = 1; +} + service ClientNamenodeProtocol { rpc getBlockLocations(GetBlockLocationsRequestProto) returns(GetBlockLocationsResponseProto); @@ -879,4 +886,6 @@ service ClientNamenodeProtocol { returns(GetEditsFromTxidResponseProto); rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto) returns(GetErasureCodingInfoResponseProto); + rpc getECSchemas(GetECSchemasRequestProto) + returns(GetECSchemasResponseProto); } 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 1314ea0718311..050753893f61f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -637,8 +637,7 @@ message ECSchemaProto { required string codecName = 2; required uint32 dataUnits = 3; required uint32 parityUnits = 4; - required uint32 chunkSize = 5; - repeated ECSchemaOptionEntryProto options = 6; + repeated ECSchemaOptionEntryProto options = 5; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java new file mode 100644 index 0000000000000..07e1359c2ddab --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java @@ -0,0 +1,57 @@ +/** + * 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 static org.junit.Assert.*; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestECSchemas { + private MiniDFSCluster cluster; + + @Before + public void before() throws IOException { + cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0) + .build(); + cluster.waitActive(); + } + + @After + public void after() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testGetECSchemas() throws Exception { + ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas(); + // TODO update assertion after HDFS-7866 + assertNotNull(ecSchemas); + assertEquals("Should have only one ecSchema", 1, ecSchemas.length); + assertEquals("Returned schemas should have only default schema", + ECSchemaManager.getSystemDefaultSchema(), ecSchemas[0]); + } +} From 35797b08899195170715835568f96ab51bef0d98 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 13 Apr 2015 10:56:24 -0700 Subject: [PATCH 059/212] HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit mainly addresses HDFS-8081 and HDFS-8048. Contributed by Zhe Zhang. --- .../org/apache/hadoop/hdfs/DFSInputStream.java | 4 ++-- .../hadoop/hdfs/DFSStripedInputStream.java | 16 +++++++++------- .../hadoop/hdfs/DFSStripedOutputStream.java | 3 ++- .../hadoop/hdfs/TestDFSStripedOutputStream.java | 3 ++- 4 files changed, 15 insertions(+), 11 deletions(-) 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 72725c43abe38..7c2dcf94b7552 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 @@ -1109,7 +1109,7 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, int offset, Map> corruptedBlockMap) throws IOException { final int length = (int) (end - start + 1); - actualGetFromOneDataNode(datanode, block, start, end, buf, + actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf, new int[]{offset}, new int[]{length}, corruptedBlockMap); } @@ -1128,7 +1128,7 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, * block replica */ void actualGetFromOneDataNode(final DNAddrPair datanode, - LocatedBlock block, final long startInBlk, final long endInBlk, + long blockStartOffset, final long startInBlk, final long endInBlk, byte[] buf, int[] offsets, int[] lengths, Map> corruptedBlockMap) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 077b0f855428e..8a431b1ba836e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -224,7 +224,7 @@ private LocatedBlock getBlockGroupAt(long offset) throws IOException { * Real implementation of pread. */ @Override - protected void fetchBlockByteRange(LocatedBlock block, long start, + protected void fetchBlockByteRange(long blockStartOffset, long start, long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { @@ -234,7 +234,7 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, int len = (int) (end - start + 1); // Refresh the striped block group - block = getBlockGroupAt(block.getStartOffset()); + LocatedBlock block = getBlockGroupAt(blockStartOffset); assert block instanceof LocatedStripedBlock : "NameNode" + " should return a LocatedStripedBlock for a striped file"; LocatedStripedBlock blockGroup = (LocatedStripedBlock) block; @@ -254,9 +254,11 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, DatanodeInfo loc = blks[i].getLocations()[0]; StorageType type = blks[i].getStorageTypes()[0]; DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( - loc.getXferAddr(dfsClient.getConf().connectToDnViaHostname)), type); - Callable readCallable = getFromOneDataNode(dnAddr, blks[i], - rp.startOffsetInBlock, rp.startOffsetInBlock + rp.readLength - 1, buf, + loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), + type); + Callable readCallable = getFromOneDataNode(dnAddr, + blks[i].getStartOffset(), rp.startOffsetInBlock, + rp.startOffsetInBlock + rp.readLength - 1, buf, rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i); Future getFromDNRequest = stripedReadsService.submit(readCallable); DFSClient.LOG.debug("Submitting striped read request for " + blks[i]); @@ -272,7 +274,7 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, } private Callable getFromOneDataNode(final DNAddrPair datanode, - final LocatedBlock block, final long start, final long end, + final long blockStartOffset, final long start, final long end, final byte[] buf, final int[] offsets, final int[] lengths, final Map> corruptedBlockMap, final int hedgedReadId) { @@ -283,7 +285,7 @@ public Void call() throws Exception { TraceScope scope = Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan); try { - actualGetFromOneDataNode(datanode, block, start, + actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf, offsets, lengths, corruptedBlockMap); } finally { scope.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index aded4fe6ef741..1d0e1beef38f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -284,7 +284,8 @@ synchronized void abort() throws IOException { } for (StripedDataStreamer streamer : streamers) { streamer.setLastException(new IOException("Lease timeout of " - + (dfsClient.getHdfsTimeout()/1000) + " seconds expired.")); + + (dfsClient.getConf().getHdfsTimeout()/1000) + + " seconds expired.")); } closeThreads(true); dfsClient.endFileLease(fileId); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index f5a37f353da50..ee6998b62d4c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -5,6 +5,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.net.TcpPeerServer; import org.apache.hadoop.hdfs.protocol.DatanodeID; @@ -241,7 +242,7 @@ private void testOneFile(String src, int writeBytes) } block.setNumBytes(lenOfBlock); - BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)). + BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)). setFileName(src). setBlock(block). setBlockToken(lblock.getBlockToken()). From a32c4dc38a719043a4f004ec174251cf0d556ab0 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 13 Apr 2015 11:08:57 -0700 Subject: [PATCH 060/212] HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of ECZone. Contributed by Vinayakumar B. --- .../org/apache/hadoop/hdfs/DFSClient.java | 6 ++-- .../hadoop/hdfs/DistributedFileSystem.java | 33 +++++++++++++++++++ .../hadoop/hdfs/protocol/ClientProtocol.java | 6 ++-- ...amenodeProtocolServerSideTranslatorPB.java | 4 ++- .../ClientNamenodeProtocolTranslatorPB.java | 5 ++- .../namenode/ErasureCodingZoneManager.java | 30 +++++++++++++---- .../hdfs/server/namenode/FSDirectory.java | 15 ++++++--- .../hdfs/server/namenode/FSNamesystem.java | 19 +++++------ .../server/namenode/NameNodeRpcServer.java | 6 ++-- .../main/proto/ClientNamenodeProtocol.proto | 1 + .../org/apache/hadoop/hdfs/DFSTestUtil.java | 2 +- .../hdfs/TestDFSStripedOutputStream.java | 2 +- .../hadoop/hdfs/TestErasureCodingZones.java | 18 +++++----- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 4 +-- .../hdfs/server/namenode/TestFSImage.java | 4 +-- 16 files changed, 108 insertions(+), 49 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 2acbf87d38647..633f38c238930 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1333,7 +1333,7 @@ public DFSOutputStream create(String src, Progressable progress, int buffersize, ChecksumOpt checksumOpt) throws IOException { - return create(src, permission, flag, createParent, replication, blockSize, + return create(src, permission, flag, createParent, replication, blockSize, progress, buffersize, checksumOpt, null); } @@ -3010,12 +3010,12 @@ public RemoteIterator listEncryptionZones() return new EncryptionZoneIterator(namenode, traceSampler); } - public void createErasureCodingZone(String src) + public void createErasureCodingZone(String src, ECSchema schema) throws IOException { checkOpen(); TraceScope scope = getPathTraceScope("createErasureCodingZone", src); try { - namenode.createErasureCodingZone(src); + namenode.createErasureCodingZone(src, schema); } catch (RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, SafeModeException.class, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 902636ce60921..bf65ff94d401e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -89,6 +89,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.Credentials; @@ -2273,4 +2274,36 @@ public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid) throws IOException { return dfs.getInotifyEventStream(lastReadTxid); } + + /** + * Create the erasurecoding zone + * + * @param path Directory to create the ec zone + * @param schema ECSchema for the zone. If not specified default will be used. + * @throws IOException + */ + public void createErasureCodingZone(final Path path, final ECSchema schema) + throws IOException { + Path absF = fixRelativePart(path); + new FileSystemLinkResolver() { + @Override + public Void doCall(final Path p) throws IOException, + UnresolvedLinkException { + dfs.createErasureCodingZone(getPathName(p), null); + return null; + } + + @Override + public Void next(final FileSystem fs, final Path p) throws IOException { + if (fs instanceof DistributedFileSystem) { + DistributedFileSystem myDfs = (DistributedFileSystem) fs; + myDfs.createErasureCodingZone(p, schema); + return null; + } + throw new UnsupportedOperationException( + "Cannot createErasureCodingZone through a symlink to a " + + "non-DistributedFileSystem: " + path + " -> " + p); + } + }.resolve(this, absF); + } } 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 3fc02014eb1fe..d48531174879b 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 @@ -1364,11 +1364,11 @@ public BatchedEntries listEncryptionZones( long prevId) throws IOException; /** - * Create an erasure coding zone (currently with hardcoded schema) - * TODO: Configurable and pluggable schemas (HDFS-7337) + * Create an erasure coding zone with specified schema, if any, otherwise + * default */ @Idempotent - public void createErasureCodingZone(String src) + public void createErasureCodingZone(String src, ECSchema schema) throws IOException; /** 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 e8f513ac1fa09..d103cf0eb5d9e 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 @@ -1403,7 +1403,9 @@ public CreateErasureCodingZoneResponseProto createErasureCodingZone( RpcController controller, CreateErasureCodingZoneRequestProto req) throws ServiceException { try { - server.createErasureCodingZone(req.getSrc()); + ECSchema schema = req.hasSchema() ? PBHelper.convertECSchema(req + .getSchema()) : null; + server.createErasureCodingZone(req.getSrc(), schema); return CreateErasureCodingZoneResponseProto.newBuilder().build(); } 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 02115227e5ae0..2e17823ef3563 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 @@ -1420,11 +1420,14 @@ public BatchedEntries listEncryptionZones(long id) } @Override - public void createErasureCodingZone(String src) + public void createErasureCodingZone(String src, ECSchema schema) throws IOException { final CreateErasureCodingZoneRequestProto.Builder builder = CreateErasureCodingZoneRequestProto.newBuilder(); builder.setSrc(src); + if (schema != null) { + builder.setSchema(PBHelper.convertECSchema(schema)); + } CreateErasureCodingZoneRequestProto req = builder.build(); try { rpcProxy.createErasureCodingZone(null, req); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 606e8048ad54d..c7daa2b958436 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -22,6 +22,9 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.IOException; import java.util.ArrayList; @@ -50,7 +53,11 @@ public ErasureCodingZoneManager(FSDirectory dir) { this.dir = dir; } - boolean getECPolicy(INodesInPath iip) { + boolean getECPolicy(INodesInPath iip) throws IOException { + return getECSchema(iip) != null; + } + + ECSchema getECSchema(INodesInPath iip) throws IOException{ assert dir.hasReadLock(); Preconditions.checkNotNull(iip); List inodes = iip.getReadOnlyINodes(); @@ -64,21 +71,23 @@ boolean getECPolicy(INodesInPath iip) { // EC // TODO: properly support symlinks in EC zones if (inode.isSymlink()) { - return false; + return null; } final List xAttrs = inode.getXAttrFeature() == null ? new ArrayList(0) : inode.getXAttrFeature().getXAttrs(); for (XAttr xAttr : xAttrs) { if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { - return true; + ECSchemaProto ecSchemaProto; + ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue()); + return PBHelper.convertECSchema(ecSchemaProto); } } } - return false; + return null; } - XAttr createErasureCodingZone(String src) + XAttr createErasureCodingZone(String src, ECSchema schema) throws IOException { assert dir.hasWriteLock(); final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false); @@ -97,8 +106,15 @@ XAttr createErasureCodingZone(String src) throw new IOException("Directory " + src + " is already in an " + "erasure coding zone."); } - final XAttr ecXAttr = XAttrHelper - .buildXAttr(XATTR_ERASURECODING_ZONE, null); + // TODO HDFS-7859 Need to persist the schema in xattr in efficient way + // As of now storing the protobuf format + if (schema == null) { + schema = ECSchemaManager.getSystemDefaultSchema(); + } + ECSchemaProto schemaProto = PBHelper.convertECSchema(schema); + byte[] schemaBytes = schemaProto.toByteArray(); + final XAttr ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE, + schemaBytes); final List xattrs = Lists.newArrayListWithCapacity(1); xattrs.add(ecXAttr); FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs, 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 af5793fa82bbe..2f96424ba4029 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 @@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.util.ByteArray; import org.apache.hadoop.hdfs.util.EnumCounters; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; @@ -388,7 +389,7 @@ void enableQuotaChecks() { void disableQuotaChecks() { skipQuotaCheck = true; } - + /** * This is a wrapper for resolvePath(). If the path passed * is prefixed with /.reserved/raw, then it checks to ensure that the caller @@ -1225,20 +1226,24 @@ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId, } } - XAttr createErasureCodingZone(String src) + XAttr createErasureCodingZone(String src, ECSchema schema) throws IOException { writeLock(); try { - return ecZoneManager.createErasureCodingZone(src); + return ecZoneManager.createErasureCodingZone(src, schema); } finally { writeUnlock(); } } - public boolean getECPolicy(INodesInPath iip) { + public boolean getECPolicy(INodesInPath iip) throws IOException { + return getECSchema(iip) != null; + } + + ECSchema getECSchema(INodesInPath iip) throws IOException { readLock(); try { - return ecZoneManager.getECPolicy(iip); + return ecZoneManager.getECSchema(iip); } finally { readUnlock(); } 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 33cbdd1453229..3a865718a6447 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 @@ -7518,16 +7518,16 @@ BatchedListEntries listEncryptionZones(long prevId) /** * Create an erasure coding zone on directory src. - * + * @param schema ECSchema for the erasure coding zone * @param src the path of a directory which will be the root of the * erasure coding zone. The directory must be empty. + * * @throws AccessControlException if the caller is not the superuser. * @throws UnresolvedLinkException if the path can't be resolved. * @throws SafeModeException if the Namenode is in safe mode. */ - void createErasureCodingZone(final String srcArg, - final boolean logRetryCache) - throws IOException, UnresolvedLinkException, + void createErasureCodingZone(final String srcArg, final ECSchema schema, + final boolean logRetryCache) throws IOException, UnresolvedLinkException, SafeModeException, AccessControlException { String src = srcArg; HdfsFileStatus resultingStat = null; @@ -7543,7 +7543,7 @@ void createErasureCodingZone(final String srcArg, checkNameNodeSafeMode("Cannot create erasure coding zone on " + src); src = dir.resolvePath(pc, src, pathComponents); - final XAttr ecXAttr = dir.createErasureCodingZone(src); + final XAttr ecXAttr = dir.createErasureCodingZone(src, schema); List xAttrs = Lists.newArrayListWithCapacity(1); xAttrs.add(ecXAttr); getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); @@ -7573,11 +7573,10 @@ ECInfo getErasureCodingInfo(String src) throws AccessControlException, if (isPermissionEnabled) { dir.checkPathAccess(pc, iip, FsAction.READ); } - if (dir.getECPolicy(iip)) { - // TODO HDFS-8074 and HDFS-7859 : To get from loaded schemas - Map options = new HashMap(); - ECSchema defaultSchema = new ECSchema("RS-6-3", "rs", 6, 3, options); - return new ECInfo(src, defaultSchema); + // Get schema set for the zone + ECSchema schema = dir.getECSchema(iip); + if (schema != null) { + return new ECInfo(src, schema); } } finally { readUnlock(); 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 d0bc453bbe594..c6b74473830b3 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 @@ -1823,8 +1823,8 @@ public BatchedEntries listEncryptionZones( } @Override // ClientProtocol - public void createErasureCodingZone(String src) - throws IOException { + public void createErasureCodingZone(String src, ECSchema schema) + throws IOException { checkNNStartup(); final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); if (cacheEntry != null && cacheEntry.isSuccess()) { @@ -1832,7 +1832,7 @@ public void createErasureCodingZone(String src) } boolean success = false; try { - namesystem.createErasureCodingZone(src, cacheEntry != null); + namesystem.createErasureCodingZone(src, schema, cacheEntry != null); } finally { RetryCache.setState(cacheEntry, success); } 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 3389a22350531..c9059bb45e472 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -716,6 +716,7 @@ message GetEditsFromTxidResponseProto { message CreateErasureCodingZoneRequestProto { required string src = 1; + optional ECSchemaProto schema = 2; } message CreateErasureCodingZoneResponseProto { 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 24943816bd306..ed508fcbf1eb7 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 @@ -1855,7 +1855,7 @@ public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, int numBlocks, int numStripesPerBlk) throws Exception { DistributedFileSystem dfs = cluster.getFileSystem(); dfs.mkdirs(dir); - dfs.getClient().createErasureCodingZone(dir.toString()); + dfs.getClient().createErasureCodingZone(dir.toString(), null); FSDataOutputStream out = null; try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index ee6998b62d4c1..c78922e47d80d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -50,7 +50,7 @@ public void setup() throws IOException { Configuration conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/"); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index bdca915b2145d..699df4e3227e3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -64,7 +64,7 @@ public void testCreateECZone() fs.mkdir(testDir, FsPermission.getDirDefault()); /* Normal creation of an erasure coding zone */ - fs.getClient().createErasureCodingZone(testDir.toString()); + fs.getClient().createErasureCodingZone(testDir.toString(), null); /* Verify files under the zone are striped */ final Path ECFilePath = new Path(testDir, "foo"); @@ -77,7 +77,7 @@ public void testCreateECZone() fs.mkdir(notEmpty, FsPermission.getDirDefault()); fs.create(new Path(notEmpty, "foo")); try { - fs.getClient().createErasureCodingZone(notEmpty.toString()); + fs.getClient().createErasureCodingZone(notEmpty.toString(), null); fail("Erasure coding zone on non-empty dir"); } catch (IOException e) { assertExceptionContains("erasure coding zone for a non-empty directory", e); @@ -87,10 +87,10 @@ public void testCreateECZone() final Path zone1 = new Path("/zone1"); final Path zone2 = new Path(zone1, "zone2"); fs.mkdir(zone1, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(zone1.toString()); + fs.getClient().createErasureCodingZone(zone1.toString(), null); fs.mkdir(zone2, FsPermission.getDirDefault()); try { - fs.getClient().createErasureCodingZone(zone2.toString()); + fs.getClient().createErasureCodingZone(zone2.toString(), null); fail("Nested erasure coding zones"); } catch (IOException e) { assertExceptionContains("already in an erasure coding zone", e); @@ -100,7 +100,7 @@ public void testCreateECZone() final Path fPath = new Path("/file"); fs.create(fPath); try { - fs.getClient().createErasureCodingZone(fPath.toString()); + fs.getClient().createErasureCodingZone(fPath.toString(), null); fail("Erasure coding zone on file"); } catch (IOException e) { assertExceptionContains("erasure coding zone for a file", e); @@ -113,8 +113,8 @@ public void testMoveValidity() throws IOException, InterruptedException { final Path dstECDir = new Path("/dstEC"); fs.mkdir(srcECDir, FsPermission.getDirDefault()); fs.mkdir(dstECDir, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(srcECDir.toString()); - fs.getClient().createErasureCodingZone(dstECDir.toString()); + fs.getClient().createErasureCodingZone(srcECDir.toString(), null); + fs.getClient().createErasureCodingZone(dstECDir.toString(), null); final Path srcFile = new Path(srcECDir, "foo"); fs.create(srcFile); @@ -158,7 +158,7 @@ public void testGetErasureCodingInfo() throws Exception { // dir ECInfo before creating ec zone assertNull(fs.getClient().getErasureCodingInfo(src)); // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src); + fs.getClient().createErasureCodingZone(src, null); verifyErasureCodingInfo(src); fs.create(new Path(ecDir, "/child1")).close(); // verify for the files in ec zone @@ -182,4 +182,4 @@ private void verifyErasureCodingInfo(String src) throws IOException { assertEquals("Default chunkSize should be used", ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize()); } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index c3c823936c887..27df1cd49d80d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -68,7 +68,7 @@ public void setup() throws IOException { .numDataNodes(GROUP_SIZE).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); - dfs.getClient().createErasureCodingZone("/"); + dfs.getClient().createErasureCodingZone("/", null); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 58c53340de43b..a5c26e920bed9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -445,7 +445,7 @@ public void testAddNewStripedBlock() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir); + fs.getClient().getNamenode().createErasureCodingZone(testDir, null); // Create a file with striped block Path p = new Path(testFilePath); @@ -517,7 +517,7 @@ public void testUpdateStripedBlocks() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir); + fs.getClient().getNamenode().createErasureCodingZone(testDir, null); //create a file with striped blocks Path p = new Path(testFilePath); 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 ead65ee4223ed..d36d78e97c914 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 @@ -135,7 +135,7 @@ private void testPersistHelper(Configuration conf) throws IOException { private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf, boolean isUC) throws IOException{ // contruct a INode with StripedBlock for saving and loading - fsn.createErasureCodingZone("/", false); + fsn.createErasureCodingZone("/", null, false); long id = 123456789; byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes(); PermissionStatus permissionStatus = new PermissionStatus("testuser_a", @@ -397,7 +397,7 @@ public void testSupportBlockGroup() throws IOException { .build(); cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); - fs.getClient().getNamenode().createErasureCodingZone("/"); + fs.getClient().getNamenode().createErasureCodingZone("/", null); Path file = new Path("/striped"); FSDataOutputStream out = fs.create(file); byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE); From 3b56a5aa9e82b5ef4fbafa832947eb80b77f0976 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 13 Apr 2015 11:15:02 -0700 Subject: [PATCH 061/212] HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this operation fails. Contributed by Rakesh R. --- .../hdfs/server/namenode/FSNamesystem.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) 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 3a865718a6447..a7d43881ef494 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 @@ -7531,11 +7531,19 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, SafeModeException, AccessControlException { String src = srcArg; HdfsFileStatus resultingStat = null; - checkSuperuserPrivilege(); - checkOperation(OperationCategory.WRITE); - final byte[][] pathComponents = - FSDirectory.getPathComponentsForReservedPath(src); - FSPermissionChecker pc = getPermissionChecker(); + FSPermissionChecker pc = null; + byte[][] pathComponents = null; + boolean success = false; + try { + checkSuperuserPrivilege(); + checkOperation(OperationCategory.WRITE); + pathComponents = + FSDirectory.getPathComponentsForReservedPath(src); + pc = getPermissionChecker(); + } catch (Throwable e) { + logAuditEvent(success, "createErasureCodingZone", srcArg); + throw e; + } writeLock(); try { checkSuperuserPrivilege(); @@ -7549,11 +7557,12 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); final INodesInPath iip = dir.getINodesInPath4Write(src, false); resultingStat = dir.getAuditFileInfo(iip); + success = true; } finally { writeUnlock(); } getEditLog().logSync(); - logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat); + logAuditEvent(success, "createErasureCodingZone", srcArg, null, resultingStat); } /** From 04c23d95d0f3a5d2e273ecd39b228f682c513714 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 15 Apr 2015 12:09:16 +0530 Subject: [PATCH 062/212] HDFS-8123. Erasure Coding: Better to move EC related proto messages to a separate erasurecoding proto file (Contrubuted by Rakesh R) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +- hadoop-hdfs-project/hadoop-hdfs/pom.xml | 1 + ...amenodeProtocolServerSideTranslatorPB.java | 12 +-- .../ClientNamenodeProtocolTranslatorPB.java | 13 ++-- .../hadoop/hdfs/protocolPB/PBHelper.java | 6 +- .../namenode/ErasureCodingZoneManager.java | 2 +- .../main/proto/ClientNamenodeProtocol.proto | 24 +----- .../src/main/proto/erasurecoding.proto | 74 +++++++++++++++++++ .../hadoop-hdfs/src/main/proto/hdfs.proto | 27 ------- 9 files changed, 96 insertions(+), 68 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 5250dfa271cec..07bbd4a59439d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -61,4 +61,7 @@ HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng) HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all - ECSchemas loaded in Namenode. (vinayakumarb) \ No newline at end of file + ECSchemas loaded in Namenode. (vinayakumarb) + + HDFS-8123. Erasure Coding: Better to move EC related proto messages to a + separate erasurecoding proto file (Rakesh R via vinayakumarb) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index 34338d1e59fc6..95fcc9844e7fe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -348,6 +348,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> hdfs.proto encryption.proto inotify.proto + erasurecoding.proto ${project.build.directory}/generated-sources/java 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 d103cf0eb5d9e..17141c36ef3d1 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 @@ -107,12 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; @@ -197,14 +193,18 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto; 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 2e17823ef3563..44d1258dab883 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 @@ -107,11 +107,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; @@ -165,13 +161,16 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.*; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; 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 c169515c8276a..10afc73f3987a 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 @@ -131,6 +131,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; @@ -151,9 +154,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index c7daa2b958436..5320c1cbaecc2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.io.erasurecode.ECSchema; 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 c9059bb45e472..89c38e904b2c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -34,6 +34,7 @@ import "acl.proto"; import "xattr.proto"; import "encryption.proto"; import "inotify.proto"; +import "erasurecoding.proto"; /** * The ClientNamenodeProtocol Service defines the interface between a client @@ -714,29 +715,6 @@ message GetEditsFromTxidResponseProto { required EventsListProto eventsList = 1; } -message CreateErasureCodingZoneRequestProto { - required string src = 1; - optional ECSchemaProto schema = 2; -} - -message CreateErasureCodingZoneResponseProto { -} - -message GetErasureCodingInfoRequestProto { - required string src = 1; -} - -message GetErasureCodingInfoResponseProto { - optional ECInfoProto ECInfo = 1; -} - -message GetECSchemasRequestProto { // void request -} - -message GetECSchemasResponseProto { - repeated ECSchemaProto schemas = 1; -} - service ClientNamenodeProtocol { rpc getBlockLocations(GetBlockLocationsRequestProto) returns(GetBlockLocationsResponseProto); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto new file mode 100644 index 0000000000000..4d5731bd279b7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -0,0 +1,74 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.hdfs.protocol.proto"; +option java_outer_classname = "ErasureCodingProtos"; +option java_generate_equals_and_hash = true; +package hadoop.hdfs; + + +/** + * ECSchema options entry + */ +message ECSchemaOptionEntryProto { + required string key = 1; + required string value = 2; +} + +/** + * ECSchema for erasurecoding + */ +message ECSchemaProto { + required string schemaName = 1; + required string codecName = 2; + required uint32 dataUnits = 3; + required uint32 parityUnits = 4; + repeated ECSchemaOptionEntryProto options = 5; +} + +/** + * ECInfo + */ +message ECInfoProto { + required string src = 1; + required ECSchemaProto schema = 2; +} + + +message CreateErasureCodingZoneRequestProto { + required string src = 1; + optional ECSchemaProto schema = 2; +} + +message CreateErasureCodingZoneResponseProto { +} + +message GetErasureCodingInfoRequestProto { + required string src = 1; +} + +message GetErasureCodingInfoResponseProto { + optional ECInfoProto ECInfo = 1; +} + +message GetECSchemasRequestProto { // void request +} + +message GetECSchemasResponseProto { + repeated ECSchemaProto schemas = 1; +} 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 050753893f61f..67e20582f2230 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -620,30 +620,3 @@ message RollingUpgradeStatusProto { required string blockPoolId = 1; optional bool finalized = 2 [default = false]; } - -/** - * ECSchema options entry - */ -message ECSchemaOptionEntryProto { - required string key = 1; - required string value = 2; -} - -/** - * ECShema for erasurecoding - */ -message ECSchemaProto { - required string schemaName = 1; - required string codecName = 2; - required uint32 dataUnits = 3; - required uint32 parityUnits = 4; - repeated ECSchemaOptionEntryProto options = 5; -} - -/** - * ECInfo - */ -message ECInfoProto { - required string src = 1; - required ECSchemaProto schema = 2; -} \ No newline at end of file From 2c277802c1635837ec926862523be02e05e5649b Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 15 Apr 2015 12:23:07 +0530 Subject: [PATCH 063/212] HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 07bbd4a59439d..9fdac987409ca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -51,11 +51,20 @@ HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones (Zhe Zhang) + HDFS-7969. Erasure coding: NameNode support for lease recovery of striped + block groups. (Zhe Zhang) + + HDFS-7782. Erasure coding: pread from files in striped layout. + (Zhe Zhang and Jing Zhao via Zhe Zhang) + HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (vinayakumarb) HDFS-8074. Define a system-wide default EC schema. (Kai Zheng) + HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. + (Jing Zhao and Zhe Zhang via Jing Zhao) + HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng) HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng) @@ -63,5 +72,11 @@ HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. (vinayakumarb) + HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of ECZone. + (Vinayakumar B via Zhe Zhang) + + HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this + operation fails. (Rakesh R via Zhe Zhang) + HDFS-8123. Erasure Coding: Better to move EC related proto messages to a separate erasurecoding proto file (Rakesh R via vinayakumarb) \ No newline at end of file From ceb3d1c17051665b67977bb5153c697239be5049 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 15 Apr 2015 16:38:22 +0530 Subject: [PATCH 064/212] HDFS-7349. Support DFS command for the EC encoding (Contributed by Vinayakumar B) --- .../java/org/apache/hadoop/fs/FsShell.java | 8 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 4 +- .../hadoop-hdfs/src/main/bin/hdfs | 5 + .../org/apache/hadoop/hdfs/DFSClient.java | 18 ++ .../hadoop/hdfs/DistributedFileSystem.java | 32 +++ .../hadoop/hdfs/protocol/ClientProtocol.java | 9 + .../hadoop/hdfs/protocol/ECZoneInfo.java | 56 +++++ ...amenodeProtocolServerSideTranslatorPB.java | 18 ++ .../ClientNamenodeProtocolTranslatorPB.java | 19 ++ .../hadoop/hdfs/protocolPB/PBHelper.java | 12 + .../namenode/ErasureCodingZoneManager.java | 11 +- .../hdfs/server/namenode/FSDirectory.java | 10 + .../hdfs/server/namenode/FSNamesystem.java | 24 ++ .../server/namenode/NameNodeRpcServer.java | 7 + .../hadoop/hdfs/tools/erasurecode/ECCli.java | 48 ++++ .../hdfs/tools/erasurecode/ECCommand.java | 209 ++++++++++++++++++ .../main/proto/ClientNamenodeProtocol.proto | 2 + .../src/main/proto/erasurecoding.proto | 15 ++ 18 files changed, 502 insertions(+), 5 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java index db73f6d17034b..f873a01ba4d16 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java @@ -111,6 +111,10 @@ public Path getCurrentTrashDir() throws IOException { return getTrash().getCurrentTrashDir(); } + protected String getUsagePrefix() { + return usagePrefix; + } + // NOTE: Usage/Help are inner classes to allow access to outer methods // that access commandFactory @@ -194,7 +198,7 @@ private void printInfo(PrintStream out, String cmd, boolean showHelp) { } } else { // display help or usage for all commands - out.println(usagePrefix); + out.println(getUsagePrefix()); // display list of short usages ArrayList instances = new ArrayList(); @@ -218,7 +222,7 @@ private void printInfo(PrintStream out, String cmd, boolean showHelp) { } private void printInstanceUsage(PrintStream out, Command instance) { - out.println(usagePrefix + " " + instance.getUsage()); + out.println(getUsagePrefix() + " " + instance.getUsage()); } private void printInstanceHelp(PrintStream out, Command instance) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9fdac987409ca..b9fc6fa260e56 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -79,4 +79,6 @@ operation fails. (Rakesh R via Zhe Zhang) HDFS-8123. Erasure Coding: Better to move EC related proto messages to a - separate erasurecoding proto file (Rakesh R via vinayakumarb) \ No newline at end of file + separate erasurecoding proto file (Rakesh R via vinayakumarb) + + HDFS-7349. Support DFS command for the EC encoding (vinayakumarb) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs index f4642617556c7..84c79b8a2f337 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs @@ -134,6 +134,11 @@ case ${COMMAND} in hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" ;; + erasurecode) + CLASS=org.apache.hadoop.hdfs.tools.erasurecode.ECCli + hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" + HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" + ;; fetchdt) CLASS=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher ;; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 633f38c238930..9c17a4ef17703 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -120,6 +120,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -3351,4 +3352,21 @@ TraceScope getSrcDstTraceScope(String description, String src, String dst) { } return scope; } + + /** + * Get the erasure coding zone information for the specified path + * + * @param src path to get the information for + * @return Returns the zone information if path is in EC Zone, null otherwise + * @throws IOException + */ + public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + checkOpen(); + try { + return namenode.getErasureCodingZoneInfo(src); + } catch (RemoteException re) { + throw re.unwrapRemoteException(FileNotFoundException.class, + AccessControlException.class, UnresolvedPathException.class); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index bf65ff94d401e..d4749f4ba209b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -75,6 +75,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -2306,4 +2307,35 @@ public Void next(final FileSystem fs, final Path p) throws IOException { } }.resolve(this, absF); } + + /** + * Get ErasureCoding zone information for the specified path + * + * @param path + * @return Returns the zone information if path is in EC zone, null otherwise + * @throws IOException + */ + public ECZoneInfo getErasureCodingZoneInfo(final Path path) + throws IOException { + Path absF = fixRelativePart(path); + return new FileSystemLinkResolver() { + @Override + public ECZoneInfo doCall(final Path p) throws IOException, + UnresolvedLinkException { + return dfs.getErasureCodingZoneInfo(getPathName(p)); + } + + @Override + public ECZoneInfo next(final FileSystem fs, final Path p) + throws IOException { + if (fs instanceof DistributedFileSystem) { + DistributedFileSystem myDfs = (DistributedFileSystem) fs; + return myDfs.getErasureCodingZoneInfo(p); + } + throw new UnsupportedOperationException( + "Cannot getErasureCodingZoneInfo through a symlink to a " + + "non-DistributedFileSystem: " + path + " -> " + p); + } + }.resolve(this, absF); + } } 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 d48531174879b..bba7697bbe56e 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 @@ -1484,4 +1484,13 @@ public List listXAttrs(String src) */ @Idempotent public ECSchema[] getECSchemas() throws IOException; + + /** + * Get the information about the EC zone for the path + * + * @param src path to get the info for + * @throws IOException + */ + @Idempotent + public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java new file mode 100644 index 0000000000000..ecfb92e789b24 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * Information about the EC Zone at the specified path. + */ +public class ECZoneInfo { + + private String dir; + private ECSchema schema; + + public ECZoneInfo(String dir, ECSchema schema) { + this.dir = dir; + this.schema = schema; + } + + /** + * Get directory of the EC zone. + * + * @return + */ + public String getDir() { + return dir; + } + + /** + * Get the schema for the EC Zone + * + * @return + */ + public ECSchema getSchema() { + return schema; + } + + @Override + public String toString() { + return "Dir: " + getDir() + ", Schema: " + schema; + } +} 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 17141c36ef3d1..f9bdb71fe6fbf 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 @@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -201,6 +202,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; @@ -1551,4 +1554,19 @@ public GetECSchemasResponseProto getECSchemas(RpcController controller, throw new ServiceException(e); } } + + @Override + public GetECZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller, + GetECZoneInfoRequestProto request) throws ServiceException { + try { + ECZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc()); + GetECZoneInfoResponseProto.Builder builder = GetECZoneInfoResponseProto.newBuilder(); + if (ecZoneInfo != null) { + builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo)); + } + return builder.build(); + } 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 44d1258dab883..6c1c9710defcc 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 @@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -167,6 +168,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; @@ -1576,4 +1579,20 @@ public ECSchema[] getECSchemas() throws IOException { throw ProtobufHelper.getRemoteException(e); } } + + @Override + public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + GetECZoneInfoRequestProto req = GetECZoneInfoRequestProto.newBuilder() + .setSrc(src).build(); + try { + GetECZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo( + null, req); + if (response.hasECZoneInfo()) { + return PBHelper.convertECZoneInfo(response.getECZoneInfo()); + } + return null; + } 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 10afc73f3987a..9ca73ae6ddca7 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 @@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -134,6 +135,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; @@ -3138,4 +3140,14 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { } return builder.build(); } + + public static ECZoneInfoProto convertECZoneInfo(ECZoneInfo ecZoneInfo) { + return ECZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir()) + .setSchema(convertECSchema(ecZoneInfo.getSchema())).build(); + } + + public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) { + return new ECZoneInfo(ecZoneInfoProto.getDir(), + convertECSchema(ecZoneInfoProto.getSchema())); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 5320c1cbaecc2..0a84083f4fac3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -57,7 +58,12 @@ boolean getECPolicy(INodesInPath iip) throws IOException { return getECSchema(iip) != null; } - ECSchema getECSchema(INodesInPath iip) throws IOException{ + ECSchema getECSchema(INodesInPath iip) throws IOException { + ECZoneInfo ecZoneInfo = getECZoneInfo(iip); + return ecZoneInfo == null ? null : ecZoneInfo.getSchema(); + } + + ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { assert dir.hasReadLock(); Preconditions.checkNotNull(iip); List inodes = iip.getReadOnlyINodes(); @@ -80,7 +86,8 @@ ECSchema getECSchema(INodesInPath iip) throws IOException{ if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { ECSchemaProto ecSchemaProto; ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue()); - return PBHelper.convertECSchema(ecSchemaProto); + ECSchema schema = PBHelper.convertECSchema(ecSchemaProto); + return new ECZoneInfo(inode.getFullPathName(), schema); } } } 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 2f96424ba4029..c09148487465a 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 @@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException; import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException; @@ -1249,6 +1250,15 @@ ECSchema getECSchema(INodesInPath iip) throws IOException { } } + ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { + readLock(); + try { + return ecZoneManager.getECZoneInfo(iip); + } finally { + readUnlock(); + } + } + static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException { INode inode = iip.getLastINode(); if (inode == null) { 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 a7d43881ef494..925f960d63fa7 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 @@ -181,6 +181,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -7593,6 +7594,29 @@ ECInfo getErasureCodingInfo(String src) throws AccessControlException, return null; } + /** + * Get the erasure coding zone information for specified path + */ + ECZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException, + UnresolvedLinkException, IOException { + checkOperation(OperationCategory.READ); + final byte[][] pathComponents = FSDirectory + .getPathComponentsForReservedPath(src); + final FSPermissionChecker pc = getPermissionChecker(); + readLock(); + try { + checkOperation(OperationCategory.READ); + src = dir.resolvePath(pc, src, pathComponents); + final INodesInPath iip = dir.getINodesInPath(src, true); + if (isPermissionEnabled) { + dir.checkPathAccess(pc, iip, FsAction.READ); + } + return dir.getECZoneInfo(iip); + } finally { + readUnlock(); + } + } + /** * Get available ECSchemas */ 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 c6b74473830b3..8217907cb7947 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 @@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FSLimitException; @@ -2043,4 +2044,10 @@ public ECSchema[] getECSchemas() throws IOException { checkNNStartup(); return namesystem.getECSchemas(); } + + @Override // ClientProtocol + public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + checkNNStartup(); + return namesystem.getErasureCodingZoneInfo(src); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java new file mode 100644 index 0000000000000..4ed9d0ae85916 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java @@ -0,0 +1,48 @@ +/** + * 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.tools.erasurecode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FsShell; +import org.apache.hadoop.fs.shell.CommandFactory; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.util.ToolRunner; + +/** + * CLI for the erasure code encoding operations. + */ +public class ECCli extends FsShell { + + private final static String usagePrefix = + "Usage: hdfs erasurecode [generic options]"; + + @Override + protected String getUsagePrefix() { + return usagePrefix; + } + + @Override + protected void registerCommands(CommandFactory factory) { + factory.registerCommands(ECCommand.class); + } + + public static void main(String[] args) throws Exception { + Configuration conf = new HdfsConfiguration(); + int res = ToolRunner.run(conf, new ECCli(), args); + System.exit(res); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java new file mode 100644 index 0000000000000..84c22751a38c1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -0,0 +1,209 @@ +/** + * 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.tools.erasurecode; + +import java.io.IOException; +import java.util.LinkedList; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.shell.Command; +import org.apache.hadoop.fs.shell.CommandFactory; +import org.apache.hadoop.fs.shell.PathData; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.util.StringUtils; + +/** + * Erasure Coding CLI commands + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public abstract class ECCommand extends Command { + + public static void registerCommands(CommandFactory factory) { + // Register all commands of Erasure CLI, with a '-' at the beginning in name + // of the command. + factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME); + factory.addClass(GetECZoneInfoCommand.class, "-" + + GetECZoneInfoCommand.NAME); + factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME); + } + + @Override + public String getCommandName() { + return getName(); + } + + @Override + protected void run(Path path) throws IOException { + throw new RuntimeException("Not suppose to get here"); + } + + @Deprecated + @Override + public int runAll() { + return run(args); + } + + @Override + protected void processPath(PathData item) throws IOException { + if (!(item.fs instanceof DistributedFileSystem)) { + throw new UnsupportedActionException( + "Erasure commands are only supported for the HDFS paths"); + } + } + + /** + * Create EC encoding zone command. Zones are created to use specific EC + * encoding schema, other than default while encoding the files under some + * specific directory. + */ + static class CreateECZoneCommand extends ECCommand { + public static final String NAME = "createZone"; + public static final String USAGE = "[-s ] "; + public static final String DESCRIPTION = + "Create a zone to encode files using a specified schema\n" + + "Options :\n" + + " -s : EC schema name to encode files. " + + "If not passed default schema will be used\n" + + " : Path to an empty directory. Under this directory " + + "files will be encoded using specified schema"; + private String schemaName; + private ECSchema schema = null; + + @Override + protected void processOptions(LinkedList args) throws IOException { + schemaName = StringUtils.popOptionWithArgument("-s", args); + if (args.isEmpty()) { + throw new HadoopIllegalArgumentException(" is missing"); + } + if (args.size() > 1) { + throw new HadoopIllegalArgumentException("Too many arguments"); + } + } + + @Override + protected void processPath(PathData item) throws IOException { + super.processPath(item); + DistributedFileSystem dfs = (DistributedFileSystem) item.fs; + try { + if (schemaName != null) { + ECSchema[] ecSchemas = dfs.getClient().getECSchemas(); + for (ECSchema ecSchema : ecSchemas) { + if (schemaName.equals(ecSchema.getSchemaName())) { + schema = ecSchema; + break; + } + } + if (schema == null) { + StringBuilder sb = new StringBuilder(); + sb.append("Schema '"); + sb.append(schemaName); + sb.append("' does not match any of the supported schemas."); + sb.append("Please select any one of ["); + for (ECSchema ecSchema : ecSchemas) { + sb.append(ecSchema.getSchemaName()); + sb.append(", "); + } + throw new HadoopIllegalArgumentException(sb.toString()); + } + } + dfs.createErasureCodingZone(item.path, schema); + out.println("EC Zone created successfully at " + item.path); + } catch (IOException e) { + throw new IOException("Unable to create EC zone for the path " + + item.path, e); + } + } + } + + /** + * Get the information about the zone + */ + static class GetECZoneInfoCommand extends ECCommand { + public static final String NAME = "getZoneInfo"; + public static final String USAGE = ""; + public static final String DESCRIPTION = + "Get information about the EC zone at specified path\n"; + + @Override + protected void processOptions(LinkedList args) throws IOException { + if (args.isEmpty()) { + throw new HadoopIllegalArgumentException(" is missing"); + } + if (args.size() > 1) { + throw new HadoopIllegalArgumentException("Too many arguments"); + } + } + + @Override + protected void processPath(PathData item) throws IOException { + super.processPath(item); + DistributedFileSystem dfs = (DistributedFileSystem) item.fs; + try { + ECZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path); + out.println(ecZoneInfo.toString()); + } catch (IOException e) { + throw new IOException("Unable to create EC zone for the path " + + item.path, e); + } + } + } + + /** + * List all supported EC Schemas + */ + static class ListECSchemas extends ECCommand { + public static final String NAME = "listSchemas"; + public static final String USAGE = ""; + public static final String DESCRIPTION = + "Get the list of ECSchemas supported\n"; + + @Override + protected void processOptions(LinkedList args) throws IOException { + if (!args.isEmpty()) { + throw new HadoopIllegalArgumentException("Too many parameters"); + } + + FileSystem fs = FileSystem.get(getConf()); + if (fs instanceof DistributedFileSystem == false) { + throw new UnsupportedActionException( + "Erasure commands are only supported for the HDFS"); + } + DistributedFileSystem dfs = (DistributedFileSystem) fs; + + ECSchema[] ecSchemas = dfs.getClient().getECSchemas(); + StringBuilder sb = new StringBuilder(); + int i = 0; + while (i < ecSchemas.length) { + ECSchema ecSchema = ecSchemas[i]; + sb.append(ecSchema.getSchemaName()); + i++; + if (i < ecSchemas.length) { + sb.append(", "); + } + } + out.println(sb.toString()); + } + } +} \ No newline at end of file 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 89c38e904b2c4..046120dcd5be5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -867,4 +867,6 @@ service ClientNamenodeProtocol { returns(GetErasureCodingInfoResponseProto); rpc getECSchemas(GetECSchemasRequestProto) returns(GetECSchemasResponseProto); + rpc getErasureCodingZoneInfo(GetECZoneInfoRequestProto) + returns(GetECZoneInfoResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 4d5731bd279b7..d888f71ef9842 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -49,6 +49,13 @@ message ECInfoProto { required ECSchemaProto schema = 2; } +/** + * ECZoneInfo + */ +message ECZoneInfoProto { + required string dir = 1; + required ECSchemaProto schema = 2; +} message CreateErasureCodingZoneRequestProto { required string src = 1; @@ -72,3 +79,11 @@ message GetECSchemasRequestProto { // void request message GetECSchemasResponseProto { repeated ECSchemaProto schemas = 1; } + +message GetECZoneInfoRequestProto { + required string src = 1; // path to get the zone info +} + +message GetECZoneInfoResponseProto { + optional ECZoneInfoProto ECZoneInfo = 1; +} \ No newline at end of file From 5e8837dd6cc747b3259751930f2a9eed7163bcad Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 15 Apr 2015 12:59:27 -0700 Subject: [PATCH 065/212] HDFS-8120. Erasure coding: created util class to analyze striped block groups. Contributed by Zhe Zhang and Li Bo. --- .../apache/hadoop/hdfs/DFSInputStream.java | 4 +- .../hadoop/hdfs/DFSStripedInputStream.java | 77 +++------- .../hadoop/hdfs/DFSStripedOutputStream.java | 34 +++-- .../hadoop/hdfs/StripedDataStreamer.java | 58 ++------ .../server/blockmanagement/BlockManager.java | 26 +++- .../hadoop/hdfs/util/StripedBlockUtil.java | 138 ++++++++++++++++++ .../org/apache/hadoop/hdfs/DFSTestUtil.java | 91 +++++++----- .../hdfs/TestDFSStripedOutputStream.java | 83 +++++------ .../hadoop/hdfs/TestReadStripedFile.java | 92 ++---------- .../server/namenode/TestAddStripedBlocks.java | 107 ++++++++++++++ .../namenode/TestRecoverStripedBlocks.java | 3 +- .../hdfs/util/TestStripedBlockUtil.java | 125 ++++++++++++++++ 12 files changed, 562 insertions(+), 276 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java 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 7c2dcf94b7552..ecf74f7abbe2e 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 @@ -1151,9 +1151,9 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, for (int i = 0; i < offsets.length; i++) { int nread = reader.readAll(buf, offsets[i], lengths[i]); updateReadStatistics(readStatistics, nread, reader); - if (nread != len) { + if (nread != lengths[i]) { throw new IOException("truncated return from reader.read(): " + - "excpected " + len + ", got " + nread); + "excpected " + lengths[i] + ", got " + nread); } } DFSClientFaultInjector.get().readFromDatanodeDelay(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 8a431b1ba836e..d597407305c1b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.net.NetUtils; import org.apache.htrace.Span; import org.apache.htrace.Trace; @@ -50,7 +51,7 @@ * * | <- Striped Block Group -> | * blk_0 blk_1 blk_2 <- A striped block group has - * | | | {@link #groupSize} blocks + * | | | {@link #dataBlkNum} blocks * v v v * +------+ +------+ +------+ * |cell_0| |cell_1| |cell_2| <- The logical read order should be @@ -72,7 +73,7 @@ public class DFSStripedInputStream extends DFSInputStream { /** * This method plans the read portion from each block in the stripe - * @param groupSize The size / width of the striping group + * @param dataBlkNum The number of data blocks in the striping group * @param cellSize The size of each striping cell * @param startInBlk Starting offset in the striped block * @param len Length of the read request @@ -81,29 +82,29 @@ public class DFSStripedInputStream extends DFSInputStream { * for an individual block in the group */ @VisibleForTesting - static ReadPortion[] planReadPortions(final int groupSize, + static ReadPortion[] planReadPortions(final int dataBlkNum, final int cellSize, final long startInBlk, final int len, int bufOffset) { - ReadPortion[] results = new ReadPortion[groupSize]; - for (int i = 0; i < groupSize; i++) { + ReadPortion[] results = new ReadPortion[dataBlkNum]; + for (int i = 0; i < dataBlkNum; i++) { results[i] = new ReadPortion(); } // cellIdxInBlk is the index of the cell in the block // E.g., cell_3 is the 2nd cell in blk_0 - int cellIdxInBlk = (int) (startInBlk / (cellSize * groupSize)); + int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum)); // blkIdxInGroup is the index of the block in the striped block group // E.g., blk_2 is the 3rd block in the group - final int blkIdxInGroup = (int) (startInBlk / cellSize % groupSize); + final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum); results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk + startInBlk % cellSize; boolean crossStripe = false; - for (int i = 1; i < groupSize; i++) { - if (blkIdxInGroup + i >= groupSize && !crossStripe) { + for (int i = 1; i < dataBlkNum; i++) { + if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) { cellIdxInBlk++; crossStripe = true; } - results[(blkIdxInGroup + i) % groupSize].startOffsetInBlock = + results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock = cellSize * cellIdxInBlk; } @@ -112,57 +113,21 @@ static ReadPortion[] planReadPortions(final int groupSize, results[blkIdxInGroup].lengths.add(firstCellLen); results[blkIdxInGroup].readLength += firstCellLen; - int i = (blkIdxInGroup + 1) % groupSize; + int i = (blkIdxInGroup + 1) % dataBlkNum; for (int done = firstCellLen; done < len; done += cellSize) { ReadPortion rp = results[i]; rp.offsetsInBuf.add(done + bufOffset); final int readLen = Math.min(len - done, cellSize); rp.lengths.add(readLen); rp.readLength += readLen; - i = (i + 1) % groupSize; + i = (i + 1) % dataBlkNum; } return results; } - /** - * This method parses a striped block group into individual blocks. - * - * @param bg The striped block group - * @param dataBlkNum the number of data blocks - * @return An array containing the blocks in the group - */ - @VisibleForTesting - static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, - int dataBlkNum, int cellSize) { - int locatedBGSize = bg.getBlockIndices().length; - // TODO not considering missing blocks for now, only identify data blocks - LocatedBlock[] lbs = new LocatedBlock[dataBlkNum]; - for (short i = 0; i < locatedBGSize; i++) { - final int idx = bg.getBlockIndices()[i]; - if (idx < dataBlkNum && lbs[idx] == null) { - lbs[idx] = constructInternalBlock(bg, i, cellSize, idx); - } - } - return lbs; - } - - private static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, - int idxInReturnedLocs, int cellSize, int idxInBlockGroup) { - final ExtendedBlock blk = new ExtendedBlock(bg.getBlock()); - blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup); - // TODO: fix the numBytes computation - - return new LocatedBlock(blk, - new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, - new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, - new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, - bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), - null); - } - - private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS; + private final short dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS; + private final short parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum) throws IOException { @@ -199,7 +164,7 @@ protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { "LocatedStripedBlock for a striped file"; int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize) - % groupSize); + % dataBlkNum); // If indexing information is returned, iterate through the index array // to find the entry for position idx in the group LocatedStripedBlock lsb = (LocatedStripedBlock) lb; @@ -213,7 +178,8 @@ protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { DFSClient.LOG.debug("getBlockAt for striped blocks, offset=" + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx); } - return constructInternalBlock(lsb, i, cellSize, idx); + return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize, + dataBlkNum, idx); } private LocatedBlock getBlockGroupAt(long offset) throws IOException { @@ -240,13 +206,14 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, LocatedStripedBlock blockGroup = (LocatedStripedBlock) block; // Planning the portion of I/O for each shard - ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, start, + ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start, len, offset); // Parse group to get chosen DN location - LocatedBlock[] blks = parseStripedBlockGroup(blockGroup, groupSize, cellSize); + LocatedBlock[] blks = StripedBlockUtil. + parseStripedBlockGroup(blockGroup, cellSize, dataBlkNum, parityBlkNum); - for (short i = 0; i < groupSize; i++) { + for (short i = 0; i < dataBlkNum; i++) { ReadPortion rp = readPortions[i]; if (rp.readLength <= 0) { continue; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 1d0e1beef38f1..f11a657d73a2f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.util.DataChecksum; @@ -309,10 +310,7 @@ protected void closeThreads(boolean force) throws IOException { streamer.closeSocket(); if (streamer.isLeadingStreamer()) { leadingStreamer = streamer; - } else { - streamer.countTailingBlockGroupBytes(); } - } catch (InterruptedException e) { throw new IOException("Failed to shutdown streamer"); } finally { @@ -320,6 +318,7 @@ protected void closeThreads(boolean force) throws IOException { setClosed(); } } + assert leadingStreamer != null : "One streamer should be leader"; leadingStreamer.countTailingBlockGroupBytes(); } @@ -337,23 +336,28 @@ public synchronized void write(byte b[], int off, int len) } private void writeParityCellsForLastStripe() throws IOException{ - if(currentBlockGroupBytes == 0 || - currentBlockGroupBytes % stripeDataSize() == 0) + long parityBlkSize = StripedBlockUtil.getInternalBlockLength( + currentBlockGroupBytes, cellSize, blockGroupDataBlocks, + blockGroupDataBlocks + 1); + if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) { return; - int lastStripeLen =(int)(currentBlockGroupBytes % stripeDataSize()); - // Size of parity cells should equal the size of the first cell, if it - // is not full. - int parityCellSize = cellSize; - int index = lastStripeLen / cellSize; - if (lastStripeLen < cellSize) { - parityCellSize = lastStripeLen; - index++; } + int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize : + (int) (parityBlkSize % cellSize); + for (int i = 0; i < blockGroupBlocks; i++) { - if (i >= index) { + long internalBlkLen = StripedBlockUtil.getInternalBlockLength( + currentBlockGroupBytes, cellSize, blockGroupDataBlocks, i); + // Pad zero bytes to make all cells exactly the size of parityCellSize + // If internal block is smaller than parity block, pad zero bytes. + // Also pad zero bytes to all parity cells + if (internalBlkLen < parityBlkSize || i >= blockGroupDataBlocks) { int position = cellBuffers[i].position(); + assert position <= parityCellSize : "If an internal block is smaller" + + " than parity block, then its last cell should be small than last" + + " parity cell"; for (int j = 0; j < parityCellSize - position; j++) { - cellBuffers[i].put((byte)0); + cellBuffers[i].put((byte) 0); } } cellBuffers[i].flip(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 710d92d7363c3..56148529ae9e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -19,16 +19,16 @@ package org.apache.hadoop.hdfs; import java.util.List; -import org.apache.hadoop.fs.StorageType; + 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.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.ByteArrayManager; -import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; @@ -134,19 +134,7 @@ void countTailingBlockGroupBytes () throws IOException { "putting a block to stripeBlocks, ie = " + ie); } } - } else if (!isParityStreamer()) { - if (block == null || block.getNumBytes() == 0) { - LocatedBlock finishedBlock = new LocatedBlock(null, null); - try { - boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30, - TimeUnit.SECONDS); - } catch (InterruptedException ie) { - //TODO: Handle InterruptedException (HDFS-7786) - ie.printStackTrace(); - } - } } - } @Override @@ -155,8 +143,10 @@ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) LocatedBlock lb = null; if (isLeadingStreamer()) { if(hasCommittedBlock) { - //when committing a block group, leading streamer has to adjust - // {@link block} including the size of block group + /** + * when committing a block group, leading streamer has to adjust + * {@link block} to include the size of block group + */ for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) { try { LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30, @@ -179,7 +169,13 @@ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) lb = super.locateFollowingBlock(excludedNodes); hasCommittedBlock = true; - LocatedBlock[] blocks = unwrapBlockGroup(lb); + assert lb instanceof LocatedStripedBlock; + DFSClient.LOG.debug("Leading streamer obtained bg " + lb); + LocatedBlock[] blocks = StripedBlockUtil. + parseStripedBlockGroup((LocatedStripedBlock) lb, + HdfsConstants.BLOCK_STRIPED_CELL_SIZE, HdfsConstants.NUM_DATA_BLOCKS, + HdfsConstants.NUM_PARITY_BLOCKS + ); assert blocks.length == blockGroupSize : "Fail to get block group from namenode: blockGroupSize: " + blockGroupSize + ", blocks.length: " + blocks.length; @@ -212,30 +208,4 @@ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) } return lb; } - - /** - * Generate other blocks in a block group according to the first one. - * - * @param firstBlockInGroup the first block in a block group - * @return other blocks in this group - */ - public static LocatedBlock[] unwrapBlockGroup( - final LocatedBlock firstBlockInGroup) { - ExtendedBlock eb = firstBlockInGroup.getBlock(); - DatanodeInfo[] locs = firstBlockInGroup.getLocations(); - String[] storageIDs = firstBlockInGroup.getStorageIDs(); - StorageType[] storageTypes = firstBlockInGroup.getStorageTypes(); - Token blockToken = firstBlockInGroup.getBlockToken(); - LocatedBlock[] blocksInGroup = new LocatedBlock[locs.length]; - for (int i = 0; i < blocksInGroup.length; i++) { - //each block in a group has the same number of bytes and timestamp - ExtendedBlock extendedBlock = new ExtendedBlock(eb.getBlockPoolId(), - eb.getBlockId() + i, eb.getNumBytes(), eb.getGenerationStamp()); - blocksInGroup[i] = new LocatedBlock(extendedBlock, - new DatanodeInfo[] {locs[i]}, new String[]{storageIDs[i]}, - new StorageType[] {storageTypes[i]}); - blocksInGroup[i].setBlockToken(blockToken); - } - return blocksInGroup; - } } 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 fd8c68492dcbf..147cc31341f91 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 @@ -83,6 +83,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; import org.apache.hadoop.net.Node; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Daemon; @@ -1974,8 +1975,8 @@ public boolean processReport(final DatanodeID nodeID, metrics.addBlockReport((int) (endTime - startTime)); } blockLog.info("BLOCK* processReport: from storage {} node {}, " + - "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage - .getStorageID(), nodeID, newReport.getNumberOfBlocks(), + "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage + .getStorageID(), nodeID, newReport.getNumberOfBlocks(), node.hasStaleStorages(), (endTime - startTime)); return !node.hasStaleStorages(); } @@ -2002,8 +2003,8 @@ private void removeZombieReplicas(BlockReportContext context, assert(zombie.numBlocks() == 0); LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " + "which no longer exists on the DataNode.", - Long.toHexString(context.getReportId()), prevBlocks, - zombie.getStorageID()); + Long.toHexString(context.getReportId()), prevBlocks, + zombie.getStorageID()); } /** @@ -2487,7 +2488,22 @@ private BlockToMarkCorrupt checkReplicaCorrupt( "block is " + ucState + " and reported genstamp " + reportedGS + " does not match genstamp in block map " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); - } else if (storedBlock.getNumBytes() != reported.getNumBytes()) { + } + boolean wrongSize; + if (storedBlock.isStriped()) { + assert BlockIdManager.isStripedBlockID(reported.getBlockId()); + assert storedBlock.getBlockId() == + BlockIdManager.convertToStripedID(reported.getBlockId()); + BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock; + int reportedBlkIdx = BlockIdManager.getBlockIndex(reported); + wrongSize = reported.getNumBytes() != + getInternalBlockLength(stripedBlock.getNumBytes(), + HdfsConstants.BLOCK_STRIPED_CELL_SIZE, + stripedBlock.getDataBlockNum(), reportedBlkIdx); + } else { + wrongSize = storedBlock.getNumBytes() != reported.getNumBytes(); + } + if (wrongSize) { return new BlockToMarkCorrupt(new Block(reported), storedBlock, "block is " + ucState + " and reported length " + reported.getNumBytes() + " does not match " + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java new file mode 100644 index 0000000000000..23680216b8dc3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -0,0 +1,138 @@ +/** + * 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.util; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.StorageType; +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.protocol.LocatedStripedBlock; + +/** + * Utility class for analyzing striped block groups + */ +@InterfaceAudience.Private +public class StripedBlockUtil { + + /** + * This method parses a striped block group into individual blocks. + * + * @param bg The striped block group + * @param cellSize The size of a striping cell + * @param dataBlkNum The number of data blocks + * @return An array containing the blocks in the group + */ + public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, + int cellSize, int dataBlkNum, int parityBlkNum) { + int locatedBGSize = bg.getBlockIndices().length; + // TODO not considering missing blocks for now, only identify data blocks + LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum]; + for (short i = 0; i < locatedBGSize; i++) { + final int idx = bg.getBlockIndices()[i]; + if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) { + lbs[idx] = constructInternalBlock(bg, i, cellSize, + dataBlkNum, idx); + } + } + return lbs; + } + + /** + * This method creates an internal block at the given index of a block group + * + * @param idxInReturnedLocs The index in the stored locations in the + * {@link LocatedStripedBlock} object + * @param idxInBlockGroup The logical index in the striped block group + * @return The constructed internal block + */ + public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, + int idxInReturnedLocs, int cellSize, int dataBlkNum, + int idxInBlockGroup) { + final ExtendedBlock blk = new ExtendedBlock(bg.getBlock()); + blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup); + blk.setNumBytes(getInternalBlockLength(bg.getBlockSize(), + cellSize, dataBlkNum, idxInBlockGroup)); + + return new LocatedBlock(blk, + new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, + new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, + new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, + bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), + null); + } + + /** + * Get the size of an internal block at the given index of a block group + * + * @param numBytesInGroup Size of the block group only counting data blocks + * @param cellSize The size of a striping cell + * @param dataBlkNum The number of data blocks + * @param idxInGroup The logical index in the striped block group + * @return The size of the internal block at the specified index + */ + public static long getInternalBlockLength(long numBytesInGroup, + int cellSize, int dataBlkNum, int idxInGroup) { + // Size of each stripe (only counting data blocks) + final long numBytesPerStripe = cellSize * dataBlkNum; + assert numBytesPerStripe > 0: + "getInternalBlockLength should only be called on valid striped blocks"; + // If block group ends at stripe boundary, each internal block has an equal + // share of the group + if (numBytesInGroup % numBytesPerStripe == 0) { + return numBytesInGroup / dataBlkNum; + } + + int numStripes = (int) ((numBytesInGroup - 1) / numBytesPerStripe + 1); + assert numStripes >= 1 : "There should be at least 1 stripe"; + + // All stripes but the last one are full stripes. The block should at least + // contain (numStripes - 1) full cells. + long blkSize = (numStripes - 1) * cellSize; + + long lastStripeLen = numBytesInGroup % numBytesPerStripe; + // Size of parity cells should equal the size of the first cell, if it + // is not full. + long lastParityCellLen = Math.min(cellSize, lastStripeLen); + + if (idxInGroup >= dataBlkNum) { + // for parity blocks + blkSize += lastParityCellLen; + } else { + // for data blocks + blkSize += Math.min(cellSize, + Math.max(0, lastStripeLen - cellSize * idxInGroup)); + } + + return blkSize; + } + + /** + * Given a byte's offset in an internal block, calculate the offset in + * the block group + */ + public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum, + long offsetInBlk, int idxInBlockGroup) { + int cellIdxInBlk = (int) (offsetInBlk / cellSize); + return cellIdxInBlk * cellSize * dataBlkNum // n full stripes before offset + + idxInBlockGroup * cellSize // m full cells before offset + + offsetInBlk % cellSize; // partial cell + } + +} 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 ed508fcbf1eb7..0c88842f9b517 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,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LayoutVersion; -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.protocol.datatransfer.Sender; @@ -1851,11 +1850,30 @@ public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock( return reports; } - public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, - int numBlocks, int numStripesPerBlk) throws Exception { + /** + * Creates the metadata of a file in striped layout. This method only + * manipulates the NameNode state without injecting data to DataNode. + * @param file Path of the file to create + * @param dir Parent path of the file + * @param numBlocks Number of striped block groups to add to the file + * @param numStripesPerBlk Number of striped cells in each block + * @param toMkdir + */ + public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir, + int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception { DistributedFileSystem dfs = cluster.getFileSystem(); - dfs.mkdirs(dir); - dfs.getClient().createErasureCodingZone(dir.toString(), null); + // If outer test already created EC zone, dir should be left as null + if (toMkdir) { + assert dir != null; + dfs.mkdirs(dir); + try { + dfs.getClient().createErasureCodingZone(dir.toString(), null); + } catch (IOException e) { + if (!e.getMessage().contains("non-empty directory")) { + throw e; + } + } + } FSDataOutputStream out = null; try { @@ -1867,7 +1885,7 @@ public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, ExtendedBlock previous = null; for (int i = 0; i < numBlocks; i++) { - Block newBlock = createBlock(cluster.getDataNodes(), dfs, ns, + Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns, file.toString(), fileNode, dfs.getClient().getClientName(), previous, numStripesPerBlk); previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock); @@ -1880,43 +1898,50 @@ public static void createECFile(MiniDFSCluster cluster, Path file, Path dir, } } - static Block createBlock(List dataNodes, DistributedFileSystem fs, - FSNamesystem ns, String file, INodeFile fileNode, String clientName, - ExtendedBlock previous, int numStripes) throws Exception { + /** + * Adds a striped block group to a file. This method only manipulates NameNode + * states of the file and the block without injecting data to DataNode. + * It does mimic block reports. + * @param dataNodes List DataNodes to host the striped block group + * @param previous Previous block in the file + * @param numStripes Number of stripes in each block group + * @return The added block group + */ + public static Block addStripedBlockToFile(List dataNodes, + DistributedFileSystem fs, FSNamesystem ns, String file, INodeFile fileNode, + String clientName, ExtendedBlock previous, int numStripes) + throws Exception { fs.getClient().namenode.addBlock(file, clientName, previous, null, fileNode.getId(), null); final BlockInfo lastBlock = fileNode.getLastBlock(); final int groupSize = fileNode.getBlockReplication(); + assert dataNodes.size() >= groupSize; // 1. RECEIVING_BLOCK IBR - int i = 0; - for (DataNode dn : dataNodes) { - if (i < groupSize) { - final Block block = new Block(lastBlock.getBlockId() + i++, 0, - lastBlock.getGenerationStamp()); - DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); - StorageReceivedDeletedBlocks[] reports = DFSTestUtil - .makeReportForReceivedBlock(block, - ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage); - for (StorageReceivedDeletedBlocks report : reports) { - ns.processIncrementalBlockReport(dn.getDatanodeId(), report); - } + for (int i = 0; i < groupSize; i++) { + DataNode dn = dataNodes.get(i); + final Block block = new Block(lastBlock.getBlockId() + i, 0, + lastBlock.getGenerationStamp()); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage); + for (StorageReceivedDeletedBlocks report : reports) { + ns.processIncrementalBlockReport(dn.getDatanodeId(), report); } } // 2. RECEIVED_BLOCK IBR - i = 0; - for (DataNode dn : dataNodes) { - if (i < groupSize) { - final Block block = new Block(lastBlock.getBlockId() + i++, - numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp()); - DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); - StorageReceivedDeletedBlocks[] reports = DFSTestUtil - .makeReportForReceivedBlock(block, - ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); - for (StorageReceivedDeletedBlocks report : reports) { - ns.processIncrementalBlockReport(dn.getDatanodeId(), report); - } + for (int i = 0; i < groupSize; i++) { + DataNode dn = dataNodes.get(i); + final Block block = new Block(lastBlock.getBlockId() + i, + numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp()); + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(block, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + for (StorageReceivedDeletedBlocks report : reports) { + ns.processIncrementalBlockReport(dn.getDatanodeId(), report); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index c78922e47d80d..4a09bda6ec122 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -2,7 +2,6 @@ import java.util.Arrays; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.client.impl.DfsClientConf; @@ -14,10 +13,12 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.token.Token; @@ -39,16 +40,16 @@ public class TestDFSStripedOutputStream { private MiniDFSCluster cluster; private Configuration conf = new Configuration(); private DistributedFileSystem fs; - int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - int blockSize = 8 * 1024 * 1024; - int cellsInBlock = blockSize / cellSize; + private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int stripesPerBlock = 4; + int blockSize = cellSize * stripesPerBlock; private int mod = 29; @Before public void setup() throws IOException { int numDNs = dataBlocks + parityBlocks + 2; Configuration conf = new Configuration(); - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); @@ -103,8 +104,7 @@ public void TestFileMoreThanOneStripe1() throws IOException { @Test public void TestFileMoreThanOneStripe2() throws IOException { - testOneFile("/MoreThanOneStripe2", - cellSize * dataBlocks * (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) + testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks + cellSize * dataBlocks + 123); } @@ -113,18 +113,22 @@ public void TestFileFullBlockGroup() throws IOException { testOneFile("/FullBlockGroup", blockSize * dataBlocks); } - //TODO: The following tests will pass after HDFS-8121 fixed -// @Test + @Test public void TestFileMoreThanABlockGroup1() throws IOException { testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); } - // @Test + @Test public void TestFileMoreThanABlockGroup2() throws IOException { - testOneFile("/MoreThanABlockGroup2", - blockSize * dataBlocks * 3 - + (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) * cellSize * dataBlocks - + 123); + testOneFile("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123); + } + + + @Test + public void TestFileMoreThanABlockGroup3() throws IOException { + testOneFile("/MoreThanABlockGroup3", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123); } private int stripeDataSize() { @@ -193,7 +197,10 @@ private void testOneFile(String src, int writeBytes) LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L); for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { - LocatedBlock[] blocks = StripedDataStreamer.unwrapBlockGroup(firstBlock); + assert firstBlock instanceof LocatedStripedBlock; + LocatedBlock[] blocks = StripedBlockUtil. + parseStripedBlockGroup((LocatedStripedBlock) firstBlock, + cellSize, dataBlocks, parityBlocks); List oneGroup = Arrays.asList(blocks); blockGroupList.add(oneGroup); } @@ -205,12 +212,6 @@ private void testOneFile(String src, int writeBytes) byte[][] dataBlockBytes = new byte[dataBlocks][]; byte[][] parityBlockBytes = new byte[allBlocks - dataBlocks][]; - //calculate the size of this block group - int lenOfBlockGroup = group < blockGroupList.size() - 1 ? - blockSize * dataBlocks : - writeBytes - blockSize * (blockGroupList.size() - 1) * dataBlocks; - int intactStripes = lenOfBlockGroup / stripeDataSize(); - int lastStripeLen = lenOfBlockGroup % stripeDataSize(); //for each block, use BlockReader to read data for (int i = 0; i < blockList.size(); i++) { @@ -223,25 +224,17 @@ private void testOneFile(String src, int writeBytes) InetSocketAddress targetAddr = NetUtils.createSocketAddr( nodes[0].getXferAddr()); - int lenOfCell = cellSize; - if (i == lastStripeLen / cellSize) { - lenOfCell = lastStripeLen % cellSize; - } else if (i > lastStripeLen / cellSize) { - lenOfCell = 0; - } - int lenOfBlock = cellSize * intactStripes + lenOfCell; - byte[] blockBytes = new byte[lenOfBlock]; + byte[] blockBytes = new byte[(int)block.getNumBytes()]; if (i < dataBlocks) { dataBlockBytes[i] = blockBytes; } else { parityBlockBytes[i - dataBlocks] = blockBytes; } - if (lenOfBlock == 0) { + if (block.getNumBytes() == 0) { continue; } - block.setNumBytes(lenOfBlock); BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)). setFileName(src). setBlock(block). @@ -276,33 +269,33 @@ public Peer newConnectedPeer(InetSocketAddress addr, } }).build(); - blockReader.readAll(blockBytes, 0, lenOfBlock); + blockReader.readAll(blockBytes, 0, (int)block.getNumBytes()); blockReader.close(); } //check if we write the data correctly - for (int i = 0; i < dataBlockBytes.length; i++) { - byte[] cells = dataBlockBytes[i]; - if (cells == null) { + for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length; blkIdxInGroup++) { + byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup]; + if (actualBlkBytes == null) { continue; } - for (int j = 0; j < cells.length; j++) { + for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) { byte expected; //calculate the postion of this byte in the file - long pos = group * dataBlocks * blockSize - + (i * cellSize + j / cellSize * cellSize * dataBlocks) - + j % cellSize; - if (pos >= writeBytes) { + long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize, + dataBlocks, posInBlk, blkIdxInGroup) + + group * blockSize * dataBlocks; + if (posInFile >= writeBytes) { expected = 0; } else { - expected = getByte(pos); + expected = getByte(posInFile); } - if (expected != cells[j]) { - Assert.fail("Unexpected byte " + cells[j] + ", expect " + expected + if (expected != actualBlkBytes[posInBlk]) { + Assert.fail("Unexpected byte " + actualBlkBytes[posInBlk] + ", expect " + expected + ". Block group index is " + group + - ", stripe index is " + j / cellSize + - ", cell index is " + i + ", byte index is " + j % cellSize); + ", stripe index is " + posInBlk / cellSize + + ", cell index is " + blkIdxInGroup + ", byte index is " + posInBlk % cellSize); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java index 849e12ea8e49a..90488c19eac66 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java @@ -21,10 +21,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; @@ -34,10 +31,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -54,17 +50,18 @@ public class TestReadStripedFile { private DistributedFileSystem fs; private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); - private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS; - private final short TOTAL_SIZE = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; + private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; + private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final int NUM_STRIPE_PER_BLOCK = 2; - private final int BLOCKSIZE = 2 * GROUP_SIZE * CELLSIZE; + private final int BLOCKSIZE = NUM_STRIPE_PER_BLOCK * DATA_BLK_NUM * CELLSIZE; @Before public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE); SimulatedFSDataset.setFactory(conf); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(TOTAL_SIZE) + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(BLK_GROUP_SIZE) .build(); cluster.waitActive(); fs = cluster.getFileSystem(); @@ -77,72 +74,14 @@ public void tearDown() { } } - private LocatedStripedBlock createDummyLocatedBlock() { - final long blockGroupID = -1048576; - DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE]; - String[] storageIDs = new String[TOTAL_SIZE]; - StorageType[] storageTypes = new StorageType[TOTAL_SIZE]; - int[] indices = new int[TOTAL_SIZE]; - for (int i = 0; i < TOTAL_SIZE; i++) { - locs[i] = new DatanodeInfo(cluster.getDataNodes().get(i).getDatanodeId()); - storageIDs[i] = cluster.getDataNodes().get(i).getDatanodeUuid(); - storageTypes[i] = StorageType.DISK; - indices[i] = (i + 2) % GROUP_SIZE; - } - return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID), - locs, storageIDs, storageTypes, indices, 0, false, null); - } - - @Test - public void testParseDummyStripedBlock() { - LocatedStripedBlock lsb = createDummyLocatedBlock(); - LocatedBlock[] blocks = DFSStripedInputStream.parseStripedBlockGroup( - lsb, GROUP_SIZE, CELLSIZE); - assertEquals(GROUP_SIZE, blocks.length); - for (int j = 0; j < GROUP_SIZE; j++) { - assertFalse(blocks[j].isStriped()); - assertEquals(j, - BlockIdManager.getBlockIndex(blocks[j].getBlock().getLocalBlock())); - assertEquals(j * CELLSIZE, blocks[j].getStartOffset()); - } - } - - @Test - public void testParseStripedBlock() throws Exception { - final int numBlocks = 4; - DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, - NUM_STRIPE_PER_BLOCK); - LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, BLOCKSIZE * numBlocks); - - assertEquals(4, lbs.locatedBlockCount()); - List lbList = lbs.getLocatedBlocks(); - for (LocatedBlock lb : lbList) { - assertTrue(lb.isStriped()); - } - - for (int i = 0; i < numBlocks; i++) { - LocatedStripedBlock lsb = (LocatedStripedBlock) (lbs.get(i)); - LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb, - GROUP_SIZE, CELLSIZE); - assertEquals(GROUP_SIZE, blks.length); - for (int j = 0; j < GROUP_SIZE; j++) { - assertFalse(blks[j].isStriped()); - assertEquals(j, - BlockIdManager.getBlockIndex(blks[j].getBlock().getLocalBlock())); - assertEquals(i * BLOCKSIZE + j * CELLSIZE, blks[j].getStartOffset()); - } - } - } - /** * Test {@link DFSStripedInputStream#getBlockAt(long)} */ @Test public void testGetBlock() throws Exception { final int numBlocks = 4; - DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, - NUM_STRIPE_PER_BLOCK); + DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks, + NUM_STRIPE_PER_BLOCK, true); LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCKSIZE * numBlocks); final DFSStripedInputStream in = @@ -151,9 +90,9 @@ public void testGetBlock() throws Exception { List lbList = lbs.getLocatedBlocks(); for (LocatedBlock aLbList : lbList) { LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList; - LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb, - GROUP_SIZE, CELLSIZE); - for (int j = 0; j < GROUP_SIZE; j++) { + LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb, + CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); + for (int j = 0; j < DATA_BLK_NUM; j++) { LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset()); assertEquals(blks[j].getBlock(), refreshed.getBlock()); assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset()); @@ -165,15 +104,16 @@ public void testGetBlock() throws Exception { @Test public void testPread() throws Exception { final int numBlocks = 4; - DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, - NUM_STRIPE_PER_BLOCK); + DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks, + NUM_STRIPE_PER_BLOCK, true); LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCKSIZE); assert lbs.get(0) instanceof LocatedStripedBlock; LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); - for (int i = 0; i < GROUP_SIZE; i++) { - Block blk = new Block(bg.getBlock().getBlockId() + i, BLOCKSIZE, + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, bg.getBlock().getGenerationStamp()); blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); cluster.injectBlocks(i, Arrays.asList(blk), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 27df1cd49d80d..6bb1162adedeb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -34,11 +34,13 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; @@ -53,6 +55,8 @@ import java.util.List; import java.util.UUID; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; import static org.junit.Assert.assertEquals; public class TestAddStripedBlocks { @@ -284,4 +288,107 @@ public void testAddUCReplica() throws Exception { Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]); } } + + @Test + public void testCheckStripedReplicaCorrupt() throws Exception { + final int numBlocks = 4; + final int numStripes = 4; + final Path filePath = new Path("/corrupt"); + final FSNamesystem ns = cluster.getNameNode().getNamesystem(); + DFSTestUtil.createStripedFile(cluster, filePath, null, + numBlocks, numStripes, false); + + INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()). + asFile(); + Assert.assertTrue(fileNode.isStriped()); + BlockInfoStriped stored = fileNode.getStripedBlocksFeature().getBlocks()[0]; + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(0, ns.getCorruptReplicaBlocks()); + + // Now send a block report with correct size + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + final Block reported = new Block(stored); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(0).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(0, ns.getCorruptReplicaBlocks()); + + // Now send a block report with wrong size + reported.setBlockId(stored.getBlockId() + 1); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE - 1); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(1).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + + // Now send a parity block report with correct size + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(2).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + + // Now send a parity block report with wrong size + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 1); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(3).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(2, ns.getCorruptReplicaBlocks()); + + // Now change the size of stored block, and test verifying the last + // block size + stored.setNumBytes(stored.getNumBytes() + 10); + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS + 2); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(3).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + + // Now send a parity block report with correct size based on adjusted + // size of stored block + /** Now stored block has {@link numStripes} full stripes + a cell + 10 */ + stored.setNumBytes(stored.getNumBytes() + BLOCK_STRIPED_CELL_SIZE); + reported.setBlockId(stored.getBlockId()); + reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(0).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + + reported.setBlockId(stored.getBlockId() + 1); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(1).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); + reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(2).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index 4292f9ad6f8e8..ea18c3ee66151 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -78,7 +78,8 @@ public void tearDown() throws Exception { @Test public void testMissingStripedBlock() throws Exception { final int numBlocks = 4; - DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, 1); + DFSTestUtil.createStripedFile(cluster, filePath, + dirPath, numBlocks, 1, true); // make sure the file is complete in NN final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java new file mode 100644 index 0000000000000..ec0b1bbdae6e2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -0,0 +1,125 @@ +/** + * 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.util; + +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class TestStripedBlockUtil { + private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; + private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; + private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + + private LocatedStripedBlock createDummyLocatedBlock() { + final long blockGroupID = -1048576; + DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_SIZE]; + String[] storageIDs = new String[BLK_GROUP_SIZE]; + StorageType[] storageTypes = new StorageType[BLK_GROUP_SIZE]; + int[] indices = new int[BLK_GROUP_SIZE]; + for (int i = 0; i < BLK_GROUP_SIZE; i++) { + indices[i] = (i + 2) % DATA_BLK_NUM; + // Location port always equal to logical index of a block, + // for easier verification + locs[i] = DFSTestUtil.getLocalDatanodeInfo(indices[i]); + storageIDs[i] = locs[i].getDatanodeUuid(); + storageTypes[i] = StorageType.DISK; + } + return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID), + locs, storageIDs, storageTypes, indices, 0, false, null); + } + + @Test + public void testParseDummyStripedBlock() { + LocatedStripedBlock lsb = createDummyLocatedBlock(); + LocatedBlock[] blocks = parseStripedBlockGroup( + lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); + assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length); + for (int i = 0; i < DATA_BLK_NUM; i++) { + assertFalse(blocks[i].isStriped()); + assertEquals(i, + BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock())); + assertEquals(i * CELLSIZE, blocks[i].getStartOffset()); + assertEquals(1, blocks[i].getLocations().length); + assertEquals(i, blocks[i].getLocations()[0].getIpcPort()); + assertEquals(i, blocks[i].getLocations()[0].getXferPort()); + } + } + + private void verifyInternalBlocks (long numBytesInGroup, long[] expected) { + for (int i = 1; i < BLK_GROUP_SIZE; i++) { + assertEquals(expected[i], + getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i)); + } + } + + @Test + public void testGetInternalBlockLength () { + // A small delta that is smaller than a cell + final int delta = 10; + assert delta < CELLSIZE; + + // Block group is smaller than a cell + verifyInternalBlocks(CELLSIZE - delta, + new long[] {CELLSIZE - delta, 0, 0, 0, 0, 0, + CELLSIZE - delta, CELLSIZE - delta, CELLSIZE - delta}); + + // Block group is exactly as large as a cell + verifyInternalBlocks(CELLSIZE, + new long[] {CELLSIZE, 0, 0, 0, 0, 0, + CELLSIZE, CELLSIZE, CELLSIZE}); + + // Block group is a little larger than a cell + verifyInternalBlocks(CELLSIZE + delta, + new long[] {CELLSIZE, delta, 0, 0, 0, 0, + CELLSIZE, CELLSIZE, CELLSIZE}); + + // Block group contains multiple stripes and ends at stripe boundary + verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE, + new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE}); + + // Block group contains multiple stripes and ends at cell boundary + // (not ending at stripe boundary) + verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE + CELLSIZE, + new long[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 3 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE}); + + // Block group contains multiple stripes and doesn't end at cell boundary + verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE - delta, + new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE}); + } + +} From 4c039b0876bb9399c2b4a751ad7b99b36349117b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 16 Apr 2015 13:16:37 -0700 Subject: [PATCH 066/212] HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. Contributed by Hui Zheng --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +- .../server/blockmanagement/BlockManager.java | 42 ++++- .../hdfs/server/namenode/FSEditLogLoader.java | 4 +- .../hdfs/server/namenode/FSImageFormat.java | 6 +- .../server/namenode/FSImageFormatPBINode.java | 2 +- .../snapshot/FSImageFormatPBSnapshot.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 106 +++++++++++ .../hdfs/server/namenode/TestFSImage.java | 169 +++++++++++++++++- 8 files changed, 321 insertions(+), 15 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index b9fc6fa260e56..78ca6d34fa090 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -81,4 +81,7 @@ HDFS-8123. Erasure Coding: Better to move EC related proto messages to a separate erasurecoding proto file (Rakesh R via vinayakumarb) - HDFS-7349. Support DFS command for the EC encoding (vinayakumarb) \ No newline at end of file + HDFS-7349. Support DFS command for the EC encoding (vinayakumarb) + + HDFS-7994. Detect if resevered EC Block ID is already used during namenode + startup. (Hui Zheng via szetszwo) 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 147cc31341f91..e149f24b32983 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 @@ -274,6 +274,9 @@ public int getPendingDataNodeMessageCount() { /** Check whether name system is running before terminating */ private boolean checkNSRunning = true; + /** Check whether there are any non-EC blocks using StripedID */ + private boolean hasNonEcBlockUsingStripedID = false; + public BlockManager(final Namesystem namesystem, final Configuration conf) throws IOException { this.namesystem = namesystem; @@ -2934,6 +2937,24 @@ public double getReplicationQueuesInitProgress() { return replicationQueuesInitProgress; } + /** + * Get the value of whether there are any non-EC blocks using StripedID. + * + * @return Returns the value of whether there are any non-EC blocks using StripedID. + */ + public boolean hasNonEcBlockUsingStripedID(){ + return hasNonEcBlockUsingStripedID; + } + + /** + * Set the value of whether there are any non-EC blocks using StripedID. + * + * @param has - the value of whether there are any non-EC blocks using StripedID. + */ + public void hasNonEcBlockUsingStripedID(boolean has){ + hasNonEcBlockUsingStripedID = has; + } + /** * Process a single possibly misreplicated block. This adds it to the * appropriate queues if necessary, and returns a result code indicating @@ -3541,8 +3562,10 @@ public BlockInfo getStoredBlock(Block block) { if (BlockIdManager.isStripedBlockID(block.getBlockId())) { info = blocksMap.getStoredBlock( new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); - } - if (info == null) { + if ((info == null) && hasNonEcBlockUsingStripedID()){ + info = blocksMap.getStoredBlock(block); + } + } else { info = blocksMap.getStoredBlock(block); } return info; @@ -3716,6 +3739,21 @@ public BlockInfo addBlockCollection(BlockInfo block, return blocksMap.addBlockCollection(block, bc); } + /** + * Do some check when adding a block to blocksmap. + * For HDFS-7994 to check whether then block is a NonEcBlockUsingStripedID. + * + */ + public BlockInfo addBlockCollectionWithCheck( + BlockInfo block, BlockCollection bc) { + if (!hasNonEcBlockUsingStripedID()){ + if (BlockIdManager.isStripedBlockID(block.getBlockId())) { + hasNonEcBlockUsingStripedID(true); + } + } + return addBlockCollection(block, bc); + } + public BlockCollection getBlockCollection(Block b) { return blocksMap.getBlockCollection(b); } 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 f546b48ef17fc..47bfa47d7bc40 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 @@ -990,7 +990,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped) newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); } - fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file); + fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file); file.addBlock(newBlockInfo); fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock); } @@ -1082,7 +1082,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, newBI = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); } - fsNamesys.getBlockManager().addBlockCollection(newBI, file); + fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file); file.addBlock(newBI); fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock); } 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 0582a0a361636..54d0d30c24ce1 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 @@ -701,7 +701,7 @@ public void updateBlocksMap(INodeFile file) { if (blocks != null) { final BlockManager bm = namesystem.getBlockManager(); for (int i = 0; i < blocks.length; i++) { - file.setBlock(i, bm.addBlockCollection(blocks[i], file)); + file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file)); } } } @@ -1006,8 +1006,8 @@ LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) { if (oldnode.numBlocks() > 0) { BlockInfo ucBlock = cons.getLastBlock(); // we do not replace the inode, just replace the last block of oldnode - BlockInfo info = namesystem.getBlockManager().addBlockCollection( - ucBlock, oldnode); + BlockInfo info = namesystem.getBlockManager() + .addBlockCollectionWithCheck(ucBlock, oldnode); oldnode.setBlock(oldnode.numBlocks() - 1, info); } 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 82398ee78bc75..583f4b07d896c 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 @@ -219,7 +219,7 @@ public static void updateBlocksMap(INodeFile file, BlockManager bm) { final BlockInfo[] blocks = file.getBlocks(); if (blocks != null) { for (int i = 0; i < blocks.length; i++) { - file.setBlock(i, bm.addBlockCollection(blocks[i], file)); + file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file)); } } } 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 74baec5f2c0de..717c8fb00becf 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 @@ -247,7 +247,7 @@ private void loadFileDiffList(InputStream in, INodeFile file, int size) (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk); if(storedBlock == null) { storedBlock = (BlockInfoContiguous) fsn.getBlockManager() - .addBlockCollection(new BlockInfoContiguous(blk, + .addBlockCollectionWithCheck(new BlockInfoContiguous(blk, copy.getFileReplication()), file); } blocks[j] = storedBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index a5c26e920bed9..57f62c9d01079 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.Assert.assertFalse; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; @@ -47,6 +48,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; @@ -570,4 +572,108 @@ public void testUpdateStripedBlocks() throws IOException{ } } } + + @Test + public void testHasNonEcBlockUsingStripedIDForAddBlock() throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/test_block_manager"; + String testFile = "testfile_addblock"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_addblock"; + String clientMachine = "testMachine_addblock"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + //check whether the hasNonEcBlockUsingStripedID is set + //after loading a addblock-editlog + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + fns.getEditLog().logAddBlock(testFilePath, file); + file.toCompleteFile(System.currentTimeMillis()); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForUpdateBlocks() + throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/test_block_manager"; + String testFile = "testfile_002"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser2"; + String clientMachine = "testMachine1"; + long blkId = 100; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + file.toCompleteFile(System.currentTimeMillis()); + + long newBlkNumBytes = 1024*8; + long newTimestamp = 1426222918+3600; + file.toUnderConstruction(clientName, clientMachine); + file.getLastBlock().setBlockId(-100); + file.getLastBlock().setNumBytes(newBlkNumBytes); + file.getLastBlock().setGenerationStamp(newTimestamp); + fns.getEditLog().logUpdateBlocks(testFilePath, file, true); + file.toCompleteFile(System.currentTimeMillis()); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + } 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 d36d78e97c914..c482f1faf2daf 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 @@ -17,11 +17,6 @@ */ package org.apache.hadoop.hdfs.server.namenode; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - import java.io.File; import java.io.DataOutput; import java.io.DataOutputStream; @@ -62,6 +57,12 @@ import org.apache.hadoop.test.PathUtils; import org.junit.Test; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + public class TestFSImage { private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ = @@ -426,4 +427,162 @@ public void testSupportBlockGroup() throws IOException { cluster.shutdown(); } } + + @Test + public void testHasNonEcBlockUsingStripedIDForLoadFile() throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/test_block_manager"; + String testFile = "testfile_loadfile"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_loadfile"; + String clientMachine = "testMachine_loadfile"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + file.toCompleteFile(System.currentTimeMillis()); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + //after nonEcBlockUsingStripedID is deleted + //the hasNonEcBlockUsingStripedID is set to false + fs = cluster.getFileSystem(); + fs.delete(p,false); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertFalse(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForLoadUCFile() + throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/test_block_manager"; + String testFile = "testfile_loaducfile"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_loaducfile"; + String clientMachine = "testMachine_loaducfile"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForLoadSnapshot() + throws IOException{ + // start a cluster + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + FSNamesystem fns = cluster.getNamesystem(); + + String testDir = "/test_block_manager"; + String testFile = "testfile_loadSnapshot"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_loadSnapshot"; + String clientMachine = "testMachine_loadSnapshot"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + Path d = new Path(testDir); + fs.mkdir(d, new FsPermission("755")); + fs.allowSnapshot(d); + + Path p = new Path(testFilePath); + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + file.toCompleteFile(System.currentTimeMillis()); + + fs.createSnapshot(d,"testHasNonEcBlockUsingStripeID"); + fs.truncate(p,0); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } } From 64f8f0a14584cdd1fbbf6da349d1e4576c419d18 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 17 Apr 2015 12:05:31 -0700 Subject: [PATCH 067/212] HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. Contributed by Hui Zheng. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 ++ .../server/blockmanagement/BlockManager.java | 18 ++++-------------- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 78ca6d34fa090..0ed61cd445e99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -85,3 +85,5 @@ HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. (Hui Zheng via szetszwo) + + HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz). 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 e149f24b32983..00ee0a7349e63 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 @@ -2946,15 +2946,6 @@ public boolean hasNonEcBlockUsingStripedID(){ return hasNonEcBlockUsingStripedID; } - /** - * Set the value of whether there are any non-EC blocks using StripedID. - * - * @param has - the value of whether there are any non-EC blocks using StripedID. - */ - public void hasNonEcBlockUsingStripedID(boolean has){ - hasNonEcBlockUsingStripedID = has; - } - /** * Process a single possibly misreplicated block. This adds it to the * appropriate queues if necessary, and returns a result code indicating @@ -3562,7 +3553,7 @@ public BlockInfo getStoredBlock(Block block) { if (BlockIdManager.isStripedBlockID(block.getBlockId())) { info = blocksMap.getStoredBlock( new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); - if ((info == null) && hasNonEcBlockUsingStripedID()){ + if ((info == null) && hasNonEcBlockUsingStripedID){ info = blocksMap.getStoredBlock(block); } } else { @@ -3746,10 +3737,9 @@ public BlockInfo addBlockCollection(BlockInfo block, */ public BlockInfo addBlockCollectionWithCheck( BlockInfo block, BlockCollection bc) { - if (!hasNonEcBlockUsingStripedID()){ - if (BlockIdManager.isStripedBlockID(block.getBlockId())) { - hasNonEcBlockUsingStripedID(true); - } + if (!hasNonEcBlockUsingStripedID && !block.isStriped() && + BlockIdManager.isStripedBlockID(block.getBlockId())) { + hasNonEcBlockUsingStripedID = true; } return addBlockCollection(block, bc); } From 909632dd9094532a2b24049069a036be0b31ea3b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 17 Apr 2015 16:07:07 -0700 Subject: [PATCH 068/212] HADOOP-11841. Remove unused ecschema-def.xml files. --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 ++ .../src/main/conf/ecschema-def.xml | 35 ------------------ .../hadoop/fs/CommonConfigurationKeys.java | 5 --- .../hadoop/io/erasurecode/SchemaLoader.java | 36 ++++++++++--------- .../io/erasurecode/TestSchemaLoader.java | 12 ++----- 5 files changed, 25 insertions(+), 65 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index b850e117b8a33..97492702216ff 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -42,3 +42,5 @@ ( Kai Zheng via vinayakumarb ) HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng) + + HADOOP-11841. Remove unused ecschema-def.xml files. (szetszwo) diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml deleted file mode 100644 index e36d38650e443..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - - - - - - 10 - 4 - RS - - \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 8a5211a70bbb9..bd2a24b022044 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -143,11 +143,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** Supported erasure codec classes */ public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs"; - public static final String IO_ERASURECODE_SCHEMA_FILE_KEY = - "io.erasurecode.schema.file"; - public static final String IO_ERASURECODE_SCHEMA_FILE_DEFAULT = - "ecschema-def.xml"; - /** Use XOR raw coder when possible for the RS codec */ public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY = "io.erasurecode.codec.rs.usexor"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java index 75dd03a0652d0..9b10c78182762 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java @@ -17,20 +17,27 @@ */ package org.apache.hadoop.io.erasurecode; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.w3c.dom.*; -import org.xml.sax.SAXException; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.util.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.w3c.dom.Text; +import org.xml.sax.SAXException; /** * A EC schema loading utility that loads predefined EC schemas from XML file @@ -42,8 +49,8 @@ public class SchemaLoader { * Load predefined ec schemas from configuration file. This file is * expected to be in the XML format. */ - public List loadSchema(Configuration conf) { - File confFile = getSchemaFile(conf); + public List loadSchema(String schemaFilePath) { + File confFile = getSchemaFile(schemaFilePath); if (confFile == null) { LOG.warn("Not found any predefined EC schema file"); return Collections.emptyList(); @@ -100,10 +107,7 @@ private List loadSchema(File schemaFile) * Path to the XML file containing predefined ec schemas. If the path is * relative, it is searched for in the classpath. */ - private File getSchemaFile(Configuration conf) { - String schemaFilePath = conf.get( - CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY, - CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_DEFAULT); + private File getSchemaFile(String schemaFilePath) { File schemaFile = new File(schemaFilePath); if (! schemaFile.isAbsolute()) { URL url = Thread.currentThread().getContextClassLoader() diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java index 7bb0a9a121857..6caeedb3b479d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java @@ -17,16 +17,14 @@ */ package org.apache.hadoop.io.erasurecode; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.junit.Test; +import static org.junit.Assert.assertEquals; import java.io.File; import java.io.FileWriter; import java.io.PrintWriter; import java.util.List; -import static org.junit.Assert.assertEquals; +import org.junit.Test; public class TestSchemaLoader { @@ -54,12 +52,8 @@ public void testLoadSchema() throws Exception { out.println(""); out.close(); - Configuration conf = new Configuration(); - conf.set(CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY, - SCHEMA_FILE); - SchemaLoader schemaLoader = new SchemaLoader(); - List schemas = schemaLoader.loadSchema(conf); + List schemas = schemaLoader.loadSchema(SCHEMA_FILE); assertEquals(2, schemas.size()); From cd458c38a0e85c8688be75f82e6f762c73777cf6 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 17 Apr 2015 17:55:19 -0700 Subject: [PATCH 069/212] HDFS-8166. DFSStripedOutputStream should not create empty blocks. Contributed by Jing Zhao. --- .../hadoop/hdfs/DFSStripedOutputStream.java | 163 ++++++++++-------- .../hadoop/hdfs/StripedDataStreamer.java | 72 +++----- .../server/blockmanagement/BlockManager.java | 17 +- .../hdfs/TestDFSStripedOutputStream.java | 162 +++++++++++------ 4 files changed, 236 insertions(+), 178 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index f11a657d73a2f..7dc00919b834f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -22,10 +22,14 @@ import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; + +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -59,12 +63,12 @@ public class DFSStripedOutputStream extends DFSOutputStream { */ private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private ByteBuffer[] cellBuffers; - private final short blockGroupBlocks = HdfsConstants.NUM_DATA_BLOCKS + private final short numAllBlocks = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; - private final short blockGroupDataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private final short numDataBlocks = HdfsConstants.NUM_DATA_BLOCKS; private int curIdx = 0; /* bytes written in current block group */ - private long currentBlockGroupBytes = 0; + //private long currentBlockGroupBytes = 0; //TODO: Use ErasureCoder interface (HDFS-7781) private RawErasureEncoder encoder; @@ -73,10 +77,6 @@ private StripedDataStreamer getLeadingStreamer() { return streamers.get(0); } - private long getBlockGroupSize() { - return blockSize * HdfsConstants.NUM_DATA_BLOCKS; - } - /** Construct a new output stream for creating a file. */ DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, EnumSet flag, Progressable progress, @@ -84,15 +84,13 @@ private long getBlockGroupSize() { throws IOException { super(dfsClient, src, stat, flag, progress, checksum, favoredNodes); DFSClient.LOG.info("Creating striped output stream"); - if (blockGroupBlocks <= 1) { - throw new IOException("The block group must contain more than one block."); - } + checkConfiguration(); - cellBuffers = new ByteBuffer[blockGroupBlocks]; + cellBuffers = new ByteBuffer[numAllBlocks]; List> stripeBlocks = new ArrayList<>(); - for (int i = 0; i < blockGroupBlocks; i++) { - stripeBlocks.add(new LinkedBlockingQueue(blockGroupBlocks)); + for (int i = 0; i < numAllBlocks; i++) { + stripeBlocks.add(new LinkedBlockingQueue(numAllBlocks)); try { cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize)); } catch (InterruptedException ie) { @@ -103,29 +101,38 @@ private long getBlockGroupSize() { } } encoder = new RSRawEncoder(); - encoder.initialize(blockGroupDataBlocks, - blockGroupBlocks - blockGroupDataBlocks, cellSize); + encoder.initialize(numDataBlocks, + numAllBlocks - numDataBlocks, cellSize); - streamers = new ArrayList<>(blockGroupBlocks); - for (short i = 0; i < blockGroupBlocks; i++) { + List s = new ArrayList<>(numAllBlocks); + for (short i = 0; i < numAllBlocks; i++) { StripedDataStreamer streamer = new StripedDataStreamer(stat, null, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager, i, stripeBlocks); if (favoredNodes != null && favoredNodes.length != 0) { streamer.setFavoredNodes(favoredNodes); } - streamers.add(streamer); + s.add(streamer); } + streamers = Collections.unmodifiableList(s); refreshStreamer(); } + private void checkConfiguration() { + if (cellSize % bytesPerChecksum != 0) { + throw new HadoopIllegalArgumentException("Invalid values: " + + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum + + ") must divide cell size (=" + cellSize + ")."); + } + } + private void refreshStreamer() { streamer = streamers.get(curIdx); } private void moveToNextStreamer() { - curIdx = (curIdx + 1) % blockGroupBlocks; + curIdx = (curIdx + 1) % numAllBlocks; refreshStreamer(); } @@ -136,20 +143,21 @@ private void moveToNextStreamer() { * @param buffers data buffers + parity buffers */ private void encode(ByteBuffer[] buffers) { - ByteBuffer[] dataBuffers = new ByteBuffer[blockGroupDataBlocks]; - ByteBuffer[] parityBuffers = new ByteBuffer[blockGroupBlocks - blockGroupDataBlocks]; - for (int i = 0; i < blockGroupBlocks; i++) { - if (i < blockGroupDataBlocks) { + ByteBuffer[] dataBuffers = new ByteBuffer[numDataBlocks]; + ByteBuffer[] parityBuffers = new ByteBuffer[numAllBlocks - numDataBlocks]; + for (int i = 0; i < numAllBlocks; i++) { + if (i < numDataBlocks) { dataBuffers[i] = buffers[i]; } else { - parityBuffers[i - blockGroupDataBlocks] = buffers[i]; + parityBuffers[i - numDataBlocks] = buffers[i]; } } encoder.encode(dataBuffers, parityBuffers); } /** - * Generate packets from a given buffer + * Generate packets from a given buffer. This is only used for streamers + * writing parity blocks. * * @param byteBuffer the given buffer to generate packets * @return packets generated @@ -185,7 +193,6 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, throw new IOException(msg); } - // If current packet has not been enqueued for transmission, // but the cell buffer is full, we need to enqueue the packet if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) { @@ -213,13 +220,13 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, //When all data cells in a stripe are ready, we need to encode //them and generate some parity cells. These cells will be //converted to packets and put to their DataStreamer's queue. - if (curIdx == blockGroupDataBlocks) { + if (curIdx == numDataBlocks) { //encode the data cells - for (int k = 0; k < blockGroupDataBlocks; k++) { + for (int k = 0; k < numDataBlocks; k++) { cellBuffers[k].flip(); } encode(cellBuffers); - for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) { + for (int i = numDataBlocks; i < numAllBlocks; i++) { ByteBuffer parityBuffer = cellBuffers[i]; List packets = generatePackets(parityBuffer); for (DFSPacket p : packets) { @@ -245,13 +252,24 @@ private int getSizeOfCellnBuffer(int cellIndex) { } private void clearCellBuffers() { - for (int i = 0; i< blockGroupBlocks; i++) { + for (int i = 0; i< numAllBlocks; i++) { cellBuffers[i].clear(); + if (i >= numDataBlocks) { + Arrays.fill(cellBuffers[i].array(), (byte) 0); + } } } private int stripeDataSize() { - return blockGroupDataBlocks * cellSize; + return numDataBlocks * cellSize; + } + + private long getCurrentBlockGroupBytes() { + long sum = 0; + for (int i = 0; i < numDataBlocks; i++) { + sum += streamers.get(i).getBytesCurBlock(); + } + return sum; } private void notSupported(String headMsg) @@ -270,7 +288,6 @@ public void hsync() throws IOException { notSupported("hsync"); } - @Override protected synchronized void start() { for (StripedDataStreamer streamer : streamers) { @@ -302,15 +319,11 @@ boolean isClosed() { // interrupt datastreamer if force is true @Override protected void closeThreads(boolean force) throws IOException { - StripedDataStreamer leadingStreamer = null; for (StripedDataStreamer streamer : streamers) { try { streamer.close(force); streamer.join(); streamer.closeSocket(); - if (streamer.isLeadingStreamer()) { - leadingStreamer = streamer; - } } catch (InterruptedException e) { throw new IOException("Failed to shutdown streamer"); } finally { @@ -318,40 +331,26 @@ protected void closeThreads(boolean force) throws IOException { setClosed(); } } - assert leadingStreamer != null : "One streamer should be leader"; - leadingStreamer.countTailingBlockGroupBytes(); - } - - @Override - public synchronized void write(int b) throws IOException { - super.write(b); - currentBlockGroupBytes = (currentBlockGroupBytes + 1) % getBlockGroupSize(); - } - - @Override - public synchronized void write(byte b[], int off, int len) - throws IOException { - super.write(b, off, len); - currentBlockGroupBytes = (currentBlockGroupBytes + len) % getBlockGroupSize(); } - private void writeParityCellsForLastStripe() throws IOException{ + private void writeParityCellsForLastStripe() throws IOException { + final long currentBlockGroupBytes = getCurrentBlockGroupBytes(); long parityBlkSize = StripedBlockUtil.getInternalBlockLength( - currentBlockGroupBytes, cellSize, blockGroupDataBlocks, - blockGroupDataBlocks + 1); + currentBlockGroupBytes, cellSize, numDataBlocks, + numDataBlocks + 1); if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) { return; } int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize : (int) (parityBlkSize % cellSize); - for (int i = 0; i < blockGroupBlocks; i++) { + for (int i = 0; i < numAllBlocks; i++) { long internalBlkLen = StripedBlockUtil.getInternalBlockLength( - currentBlockGroupBytes, cellSize, blockGroupDataBlocks, i); + currentBlockGroupBytes, cellSize, numDataBlocks, i); // Pad zero bytes to make all cells exactly the size of parityCellSize // If internal block is smaller than parity block, pad zero bytes. // Also pad zero bytes to all parity cells - if (internalBlkLen < parityBlkSize || i >= blockGroupDataBlocks) { + if (internalBlkLen < parityBlkSize || i >= numDataBlocks) { int position = cellBuffers[i].position(); assert position <= parityCellSize : "If an internal block is smaller" + " than parity block, then its last cell should be small than last" + @@ -365,9 +364,9 @@ private void writeParityCellsForLastStripe() throws IOException{ encode(cellBuffers); //write parity cells - curIdx = blockGroupDataBlocks; + curIdx = numDataBlocks; refreshStreamer(); - for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) { + for (int i = numDataBlocks; i < numAllBlocks; i++) { ByteBuffer parityBuffer = cellBuffers[i]; List packets = generatePackets(parityBuffer); for (DFSPacket p : packets) { @@ -385,7 +384,7 @@ private void writeParityCellsForLastStripe() throws IOException{ @Override void setClosed() { super.setClosed(); - for (int i = 0; i < blockGroupBlocks; i++) { + for (int i = 0; i < numAllBlocks; i++) { byteArrayManager.release(cellBuffers[i].array()); streamers.get(i).release(); } @@ -395,10 +394,11 @@ void setClosed() { protected synchronized void closeImpl() throws IOException { if (isClosed()) { IOException e = getLeadingStreamer().getLastException().getAndSet(null); - if (e == null) - return; - else + if (e != null) { throw e; + } else { + return; + } } try { @@ -408,14 +408,13 @@ protected synchronized void closeImpl() throws IOException { streamer.waitAndQueuePacket(currentPacket); currentPacket = null; } - //if the last stripe is incomplete, generate and write parity cells + // if the last stripe is incomplete, generate and write parity cells writeParityCellsForLastStripe(); - for (int i = 0; i < blockGroupBlocks; i++) { + for (int i = 0; i < numAllBlocks; i++) { curIdx = i; refreshStreamer(); - if (streamer.getBytesCurBlock()!= 0 || - currentBlockGroupBytes < getBlockGroupSize()) { + if (streamer.getBytesCurBlock() > 0) { // send an empty packet to mark the end of the block currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), streamer.getAndIncCurrentSeqno(), true); @@ -425,9 +424,8 @@ currentBlockGroupBytes < getBlockGroupSize()) { flushInternal(); } - // get last block before destroying the streamer - ExtendedBlock lastBlock = streamers.get(0).getBlock(); closeThreads(false); + final ExtendedBlock lastBlock = getCommittedBlock(); TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER); try { completeFile(lastBlock); @@ -435,10 +433,35 @@ currentBlockGroupBytes < getBlockGroupSize()) { scope.close(); } dfsClient.endFileLease(fileId); - } catch (ClosedChannelException e) { + } catch (ClosedChannelException ignored) { } finally { setClosed(); } } + /** + * Generate the block which is reported and will be committed in NameNode. + * Need to go through all the streamers writing data blocks and add their + * bytesCurBlock together. Note that at this time all streamers have been + * closed. Also this calculation can cover streamers with writing failures. + * + * @return An ExtendedBlock with size of the whole block group. + */ + ExtendedBlock getCommittedBlock() throws IOException { + ExtendedBlock b = getLeadingStreamer().getBlock(); + if (b == null) { + return null; + } + final ExtendedBlock block = new ExtendedBlock(b); + final boolean atBlockGroupBoundary = + getLeadingStreamer().getBytesCurBlock() == 0 && + getLeadingStreamer().getBlock() != null && + getLeadingStreamer().getBlock().getNumBytes() > 0; + for (int i = 1; i < numDataBlocks; i++) { + block.setNumBytes(block.getNumBytes() + + (atBlockGroupBoundary ? streamers.get(i).getBlock().getNumBytes() : + streamers.get(i).getBytesCurBlock())); + } + return block; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 56148529ae9e8..19c205e27b8ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -22,7 +22,6 @@ 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.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; @@ -37,6 +36,10 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; + /**************************************************************************** * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}. * There are two kinds of StripedDataStreamer, leading streamer and ordinary @@ -47,9 +50,7 @@ ****************************************************************************/ public class StripedDataStreamer extends DataStreamer { private final short index; - private final List> stripedBlocks; - private static short blockGroupSize = HdfsConstants.NUM_DATA_BLOCKS - + HdfsConstants.NUM_PARITY_BLOCKS; + private final List> stripedBlocks; private boolean hasCommittedBlock = false; StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block, @@ -88,66 +89,38 @@ public boolean isLeadingStreamer () { } private boolean isParityStreamer() { - return index >= HdfsConstants.NUM_DATA_BLOCKS; + return index >= NUM_DATA_BLOCKS; } @Override protected void endBlock() { if (!isLeadingStreamer() && !isParityStreamer()) { - //before retrieving a new block, transfer the finished block to - //leading streamer + // before retrieving a new block, transfer the finished block to + // leading streamer LocatedBlock finishedBlock = new LocatedBlock( new ExtendedBlock(block.getBlockPoolId(), block.getBlockId(), - block.getNumBytes(),block.getGenerationStamp()), null); - try{ + block.getNumBytes(), block.getGenerationStamp()), null); + try { boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30, TimeUnit.SECONDS); - }catch (InterruptedException ie) { - //TODO: Handle InterruptedException (HDFS-7786) + } catch (InterruptedException ie) { + // TODO: Handle InterruptedException (HDFS-7786) } } super.endBlock(); } - /** - * This function is called after the streamer is closed. - */ - void countTailingBlockGroupBytes () throws IOException { - if (isLeadingStreamer()) { - //when committing a block group, leading streamer has to adjust - // {@link block} including the size of block group - for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) { - try { - LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30, - TimeUnit.SECONDS); - if (finishedLocatedBlock == null) { - throw new IOException("Fail to get finished LocatedBlock " + - "from streamer, i=" + i); - } - ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock(); - long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes(); - if (block != null) { - block.setNumBytes(block.getNumBytes() + bytes); - } - } catch (InterruptedException ie) { - DFSClient.LOG.info("InterruptedException received when " + - "putting a block to stripeBlocks, ie = " + ie); - } - } - } - } - @Override protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) throws IOException { LocatedBlock lb = null; if (isLeadingStreamer()) { - if(hasCommittedBlock) { + if (hasCommittedBlock) { /** * when committing a block group, leading streamer has to adjust * {@link block} to include the size of block group */ - for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) { + for (int i = 1; i < NUM_DATA_BLOCKS; i++) { try { LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30, TimeUnit.SECONDS); @@ -157,7 +130,7 @@ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) } ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock(); long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes(); - if(block != null) { + if (block != null) { block.setNumBytes(block.getNumBytes() + bytes); } } catch (InterruptedException ie) { @@ -171,14 +144,13 @@ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) hasCommittedBlock = true; assert lb instanceof LocatedStripedBlock; DFSClient.LOG.debug("Leading streamer obtained bg " + lb); - LocatedBlock[] blocks = StripedBlockUtil. - parseStripedBlockGroup((LocatedStripedBlock) lb, - HdfsConstants.BLOCK_STRIPED_CELL_SIZE, HdfsConstants.NUM_DATA_BLOCKS, - HdfsConstants.NUM_PARITY_BLOCKS - ); - assert blocks.length == blockGroupSize : + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) lb, BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, + NUM_PARITY_BLOCKS); + assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) : "Fail to get block group from namenode: blockGroupSize: " + - blockGroupSize + ", blocks.length: " + blocks.length; + (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " + + blocks.length; lb = blocks[0]; for (int i = 1; i < blocks.length; i++) { try { @@ -199,7 +171,7 @@ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) } } else { try { - //wait 90 seconds to get a block from the queue + // wait 90 seconds to get a block from the queue lb = stripedBlocks.get(index).poll(90, TimeUnit.SECONDS); } catch (InterruptedException ie) { DFSClient.LOG.info("InterruptedException received when retrieving " + 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 00ee0a7349e63..90507e99bcf69 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 @@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -598,8 +599,20 @@ public int getDefaultStorageNum(BlockInfo block) { } public short getMinStorageNum(BlockInfo block) { - return block.isStriped() ? - ((BlockInfoStriped) block).getDataBlockNum() : minReplication; + if (block.isStriped()) { + final BlockInfoStriped sblock = (BlockInfoStriped) block; + short dataBlockNum = sblock.getDataBlockNum(); + if (sblock.isComplete() || + sblock.getBlockUCState() == BlockUCState.COMMITTED) { + // if the sblock is committed/completed and its length is less than a + // full stripe, the minimum storage number needs to be adjusted + dataBlockNum = (short) Math.min(dataBlockNum, + (sblock.getNumBytes() - 1) / HdfsConstants.BLOCK_STRIPED_CELL_SIZE + 1); + } + return dataBlockNum; + } else { + return minReplication; + } } public boolean hasMinStorage(BlockInfo block) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 4a09bda6ec122..cc20f4055b3d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -1,5 +1,6 @@ package org.apache.hadoop.hdfs; +import java.nio.ByteBuffer; import java.util.Arrays; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -20,6 +21,8 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.token.Token; import org.junit.After; @@ -42,8 +45,8 @@ public class TestDFSStripedOutputStream { private DistributedFileSystem fs; private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final int stripesPerBlock = 4; - int blockSize = cellSize * stripesPerBlock; - private int mod = 29; + private final int blockSize = cellSize * stripesPerBlock; + private final RawErasureEncoder encoder = new RSRawEncoder(); @Before public void setup() throws IOException { @@ -53,6 +56,7 @@ public void setup() throws IOException { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); + encoder.initialize(dataBlocks, parityBlocks, cellSize); } @After @@ -144,60 +148,27 @@ private byte[] generateBytes(int cnt) { } private byte getByte(long pos) { + int mod = 29; return (byte) (pos % mod + 1); } - private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) - throws IOException { - Path TestPath = new Path(src); - byte[] bytes = generateBytes(writeBytes); - DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); - - //check file length - FileStatus status = fs.getFileStatus(TestPath); - long fileLength = status.getLen(); - if (fileLength != writeBytes) { - Assert.fail("File Length error: expect=" + writeBytes - + ", actual=" + fileLength); - } - - DFSStripedInputStream dis = new DFSStripedInputStream( - fs.getClient(), src, true); - byte[] buf = new byte[writeBytes + 100]; - int readLen = dis.read(0, buf, 0, buf.length); - readLen = readLen >= 0 ? readLen : 0; - if (readLen != writeBytes) { - Assert.fail("The length of file is not correct."); - } - - for (int i = 0; i < writeBytes; i++) { - if (getByte(i) != buf[i]) { - Assert.fail("Byte at i = " + i + " is wrongly written."); - } - } - } - private void testOneFile(String src, int writeBytes) throws IOException { - Path TestPath = new Path(src); + Path testPath = new Path(src); - int allBlocks = dataBlocks + parityBlocks; byte[] bytes = generateBytes(writeBytes); - DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); + DFSTestUtil.writeFile(fs, testPath, new String(bytes)); - //check file length - FileStatus status = fs.getFileStatus(TestPath); + // check file length + FileStatus status = fs.getFileStatus(testPath); long fileLength = status.getLen(); - if (fileLength != writeBytes) { - Assert.fail("File Length error: expect=" + writeBytes - + ", actual=" + fileLength); - } + Assert.assertEquals(writeBytes, fileLength); List> blockGroupList = new ArrayList<>(); LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L); for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { - assert firstBlock instanceof LocatedStripedBlock; + Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); LocatedBlock[] blocks = StripedBlockUtil. parseStripedBlockGroup((LocatedStripedBlock) firstBlock, cellSize, dataBlocks, parityBlocks); @@ -205,15 +176,14 @@ private void testOneFile(String src, int writeBytes) blockGroupList.add(oneGroup); } - //test each block group + // test each block group for (int group = 0; group < blockGroupList.size(); group++) { //get the data of this block List blockList = blockGroupList.get(group); byte[][] dataBlockBytes = new byte[dataBlocks][]; - byte[][] parityBlockBytes = new byte[allBlocks - dataBlocks][]; + byte[][] parityBlockBytes = new byte[parityBlocks][]; - - //for each block, use BlockReader to read data + // for each block, use BlockReader to read data for (int i = 0; i < blockList.size(); i++) { LocatedBlock lblock = blockList.get(i); if (lblock == null) { @@ -269,19 +239,20 @@ public Peer newConnectedPeer(InetSocketAddress addr, } }).build(); - blockReader.readAll(blockBytes, 0, (int)block.getNumBytes()); + blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); blockReader.close(); } - //check if we write the data correctly - for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length; blkIdxInGroup++) { - byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup]; + // check if we write the data correctly + for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length; + blkIdxInGroup++) { + final byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup]; if (actualBlkBytes == null) { continue; } for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) { byte expected; - //calculate the postion of this byte in the file + // calculate the position of this byte in the file long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize, dataBlocks, posInBlk, blkIdxInGroup) + group * blockSize * dataBlocks; @@ -291,15 +262,94 @@ public Peer newConnectedPeer(InetSocketAddress addr, expected = getByte(posInFile); } - if (expected != actualBlkBytes[posInBlk]) { - Assert.fail("Unexpected byte " + actualBlkBytes[posInBlk] + ", expect " + expected - + ". Block group index is " + group + - ", stripe index is " + posInBlk / cellSize + - ", cell index is " + blkIdxInGroup + ", byte index is " + posInBlk % cellSize); + String s = "Unexpected byte " + actualBlkBytes[posInBlk] + + ", expect " + expected + + ". Block group index is " + group + + ", stripe index is " + posInBlk / cellSize + + ", cell index is " + blkIdxInGroup + + ", byte index is " + posInBlk % cellSize; + Assert.assertEquals(s, expected, actualBlkBytes[posInBlk]); + } + } + + // verify the parity blocks + final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks]; + final long groupSize = lbs.getLocatedBlocks().get(group).getBlockSize(); + int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(groupSize, + cellSize, dataBlocks, dataBlocks); + for (int i = 0; i < parityBlocks; i++) { + parityBuffers[i] = ByteBuffer.allocate(parityBlkSize); + } + final int numStripes = (int) (groupSize - 1) / stripeDataSize() + 1; + for (int i = 0; i < numStripes; i++) { + final int parityCellSize = i < numStripes - 1 || parityBlkSize % cellSize == 0 + ? cellSize : parityBlkSize % cellSize; + ByteBuffer[] stripeBuf = new ByteBuffer[dataBlocks]; + for (int k = 0; k < stripeBuf.length; k++) { + stripeBuf[k] = ByteBuffer.allocate(cellSize); + } + for (int j = 0; j < dataBlocks; j++) { + if (dataBlockBytes[j] != null) { + int length = Math.min(cellSize, + dataBlockBytes[j].length - cellSize * i); + if (length > 0) { + stripeBuf[j].put(dataBlockBytes[j], cellSize * i, length); + } + } + final long pos = stripeBuf[j].position(); + for (int k = 0; k < parityCellSize - pos; k++) { + stripeBuf[j].put((byte) 0); } + stripeBuf[j].flip(); } + ByteBuffer[] parityBuf = new ByteBuffer[parityBlocks]; + for (int j = 0; j < parityBlocks; j++) { + parityBuf[j] = ByteBuffer.allocate(cellSize); + for (int k = 0; k < parityCellSize; k++) { + parityBuf[j].put((byte) 0); + } + parityBuf[j].flip(); + } + + encoder.encode(stripeBuf, parityBuf); + for (int j = 0; j < parityBlocks; j++) { + parityBuffers[j].put(parityBuf[j]); + } + } + + for (int i = 0; i < parityBlocks; i++) { + Assert.assertArrayEquals(parityBuffers[i].array(), parityBlockBytes[i]); } } } + private void testReadWriteOneFile(String src, int writeBytes) + throws IOException { + Path TestPath = new Path(src); + byte[] bytes = generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); + + //check file length + FileStatus status = fs.getFileStatus(TestPath); + long fileLength = status.getLen(); + if (fileLength != writeBytes) { + Assert.fail("File Length error: expect=" + writeBytes + + ", actual=" + fileLength); + } + + DFSStripedInputStream dis = new DFSStripedInputStream( + fs.getClient(), src, true); + byte[] buf = new byte[writeBytes + 100]; + int readLen = dis.read(0, buf, 0, buf.length); + readLen = readLen >= 0 ? readLen : 0; + if (readLen != writeBytes) { + Assert.fail("The length of file is not correct."); + } + + for (int i = 0; i < writeBytes; i++) { + if (getByte(i) != buf[i]) { + Assert.fail("Byte at i = " + i + " is wrongly written."); + } + } + } } From ab76e1fe36b52f291c70630ca1cd0861b74fc20f Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 17 Apr 2015 18:07:07 -0700 Subject: [PATCH 070/212] HDFS-7937. Erasure Coding: INodeFile quota computation unit tests. Contributed by Kai Sasaki. --- .../blockmanagement/BlockInfoStriped.java | 23 +- .../server/namenode/TestStripedINodeFile.java | 229 ++++++++++++++++++ 2 files changed, 250 insertions(+), 2 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 20b0c5c4a27c6..9f2f5ba5f8eaf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -203,8 +203,27 @@ public long spaceConsumed() { // In case striped blocks, total usage by this striped blocks should // be the total of data blocks and parity blocks because // `getNumBytes` is the total of actual data block size. - return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CELL_SIZE) + 1) - * BLOCK_STRIPED_CELL_SIZE * parityBlockNum + getNumBytes(); + + // 0. Calculate the total bytes per stripes + long numBytesPerStripe = dataBlockNum * BLOCK_STRIPED_CELL_SIZE; + if (getNumBytes() % numBytesPerStripe == 0) { + return getNumBytes() / dataBlockNum * getTotalBlockNum(); + } + // 1. Calculate the number of stripes in this block group. + long numStripes = (getNumBytes() - 1) / numBytesPerStripe + 1; + // 2. Calculate the parity cell length in the last stripe. Note that the + // size of parity cells should equal the size of the first cell, if it + // is not full. + long lastStripeParityCellLen = Math.min(getNumBytes() % numBytesPerStripe, + BLOCK_STRIPED_CELL_SIZE); + // 3. Total consumed space is the total of + // - The total of the full cells of data blocks and parity blocks. + // - The remaining of data block which does not make a stripe. + // - The last parity block cells. These size should be same + // to the first cell in this stripe. + return getTotalBlockNum() * (BLOCK_STRIPED_CELL_SIZE * (numStripes - 1)) + + getNumBytes() % numBytesPerStripe + + lastStripeParityCellLen * parityBlockNum; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java new file mode 100644 index 0000000000000..d251c30ad86cd --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -0,0 +1,229 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; +import org.apache.hadoop.io.erasurecode.ECSchema; + +import org.junit.Test; + +/** + * This class tests INodeFile with striped feature. + */ +public class TestStripedINodeFile { + public static final Log LOG = LogFactory.getLog(TestINodeFile.class); + + private static final PermissionStatus perm = new PermissionStatus( + "userName", null, FsPermission.getDefault()); + + private static INodeFile createStripedINodeFile() { + return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, + null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID); + } + + @Test + public void testBlockStripedFeature() + throws IOException, InterruptedException{ + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + assertTrue(inf.isStriped()); + } + + @Test + public void testBlockStripedTotalBlockCount() { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + assertEquals(9, blockInfoStriped.getTotalBlockNum()); + } + + @Test + public void testBlockStripedLength() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + inf.addBlock(blockInfoStriped); + assertEquals(1, inf.getBlocks().length); + } + + @Test + public void testBlockStripedConsumedSpace() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + blockInfoStriped.setNumBytes(1); + inf.addBlock(blockInfoStriped); + // 0. Calculate the total bytes per stripes + // 1. Calculate the number of stripes in this block group. + // 2. Calculate the last remaining length which does not make a stripe. + // 3. Total consumed space is the total of + // a. The total of the full cells of data blocks and parity blocks. + // b. The remaining of data block which does not make a stripe. + // c. The last parity block cells. These size should be same + // to the first cell in this stripe. + // So the total consumed space is the sum of + // a. * ( - 1) * = 0 + // b. % = 1 + // c. * = 1 * 3 + assertEquals(4, inf.storagespaceConsumedWithStriped()); + assertEquals(4, inf.storagespaceConsumed()); + } + + @Test + public void testMultipleBlockStripedConsumedSpace() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk1 = new Block(1); + BlockInfoStriped blockInfoStriped1 + = new BlockInfoStriped(blk1, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + blockInfoStriped1.setNumBytes(1); + Block blk2 = new Block(2); + BlockInfoStriped blockInfoStriped2 + = new BlockInfoStriped(blk2, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + blockInfoStriped2.setNumBytes(1); + inf.addBlock(blockInfoStriped1); + inf.addBlock(blockInfoStriped2); + // This is the double size of one block in above case. + assertEquals(4 * 2, inf.storagespaceConsumedWithStriped()); + assertEquals(4 * 2, inf.storagespaceConsumed()); + } + + @Test + public void testBlockStripedFileSize() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + blockInfoStriped.setNumBytes(100); + inf.addBlock(blockInfoStriped); + // Compute file size should return actual data + // size which is retained by this file. + assertEquals(100, inf.computeFileSize()); + assertEquals(100, inf.computeFileSize(false, false)); + } + + @Test + public void testBlockStripedUCFileSize() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk = new Block(1); + BlockInfoStripedUnderConstruction bInfoStripedUC + = new BlockInfoStripedUnderConstruction(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + bInfoStripedUC.setNumBytes(100); + inf.addBlock(bInfoStripedUC); + assertEquals(100, inf.computeFileSize()); + assertEquals(0, inf.computeFileSize(false, false)); + } + + @Test + public void testBlockStripedComputeQuotaUsage() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + blockInfoStriped.setNumBytes(100); + inf.addBlock(blockInfoStriped); + + BlockStoragePolicySuite suite = + BlockStoragePolicySuite.createDefaultSuite(); + QuotaCounts counts = + inf.computeQuotaUsageWithStriped(suite, + new QuotaCounts.Builder().build()); + assertEquals(1, counts.getNameSpace()); + // The total consumed space is the sum of + // a. * ( - 1) * = 0 + // b. % = 100 + // c. * = 100 * 3 + assertEquals(400, counts.getStorageSpace()); + } + + @Test + public void testBlockStripedUCComputeQuotaUsage() + throws IOException, InterruptedException { + ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + INodeFile inf = createStripedINodeFile(); + inf.addStripedBlocksFeature(); + Block blk = new Block(1); + BlockInfoStripedUnderConstruction bInfoStripedUC + = new BlockInfoStripedUnderConstruction(blk, + (short)defaultSchema.getNumDataUnits(), + (short)defaultSchema.getNumParityUnits()); + bInfoStripedUC.setNumBytes(100); + inf.addBlock(bInfoStripedUC); + + BlockStoragePolicySuite suite + = BlockStoragePolicySuite.createDefaultSuite(); + QuotaCounts counts + = inf.computeQuotaUsageWithStriped(suite, + new QuotaCounts.Builder().build()); + assertEquals(1024, inf.getPreferredBlockSize()); + assertEquals(1, counts.getNameSpace()); + // Consumed space in the case of BlockInfoStripedUC can be calculated + // by using preferred block size. This is 1024 and total block num + // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216. + assertEquals(9216, counts.getStorageSpace()); + } +} From f6e1160ef1e946a5f6c9503b06832e6b84c36edb Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 17 Apr 2015 18:13:47 -0700 Subject: [PATCH 071/212] HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks. Contributed by Jing Zhao. --- .../blockmanagement/BlockInfoStriped.java | 7 ------ .../namenode/ErasureCodingZoneManager.java | 12 ++++------ .../hdfs/server/namenode/FSDirectory.java | 2 +- .../hdfs/server/namenode/FSEditLogLoader.java | 14 ++++++----- .../hdfs/server/namenode/FSImageFormat.java | 4 +--- .../server/namenode/FSImageSerialization.java | 13 +++++------ .../blockmanagement/TestBlockInfoStriped.java | 23 ++++++------------- .../hdfs/server/namenode/TestFSImage.java | 2 +- 8 files changed, 29 insertions(+), 48 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 9f2f5ba5f8eaf..23e3153b57807 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -244,13 +244,6 @@ public int numNodes() { return num; } - @Override - public void write(DataOutput out) throws IOException { - out.writeShort(dataBlockNum); - out.writeShort(parityBlockNum); - super.write(out); - } - /** * Convert a complete block to an under construction block. * @return BlockInfoUnderConstruction - an under construction block. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 0a84083f4fac3..3f942279024d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -54,10 +54,6 @@ public ErasureCodingZoneManager(FSDirectory dir) { this.dir = dir; } - boolean getECPolicy(INodesInPath iip) throws IOException { - return getECSchema(iip) != null; - } - ECSchema getECSchema(INodesInPath iip) throws IOException { ECZoneInfo ecZoneInfo = getECZoneInfo(iip); return ecZoneInfo == null ? null : ecZoneInfo.getSchema(); @@ -109,7 +105,7 @@ XAttr createErasureCodingZone(String src, ECSchema schema) throw new IOException("Attempt to create an erasure coding zone " + "for a file."); } - if (getECPolicy(srcIIP)) { + if (getECSchema(srcIIP) != null) { throw new IOException("Directory " + src + " is already in an " + "erasure coding zone."); } @@ -132,8 +128,10 @@ XAttr createErasureCodingZone(String src, ECSchema schema) void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src) throws IOException { assert dir.hasReadLock(); - if (getECPolicy(srcIIP) - != getECPolicy(dstIIP)) { + final ECSchema srcSchema = getECSchema(srcIIP); + final ECSchema dstSchema = getECSchema(dstIIP); + if ((srcSchema != null && !srcSchema.equals(dstSchema)) || + (dstSchema != null && !dstSchema.equals(srcSchema))) { throw new IOException( src + " can't be moved because the source and destination have " + "different erasure coding policies."); 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 c09148487465a..7392552133fd0 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 @@ -1237,7 +1237,7 @@ XAttr createErasureCodingZone(String src, ECSchema schema) } } - public boolean getECPolicy(INodesInPath iip) throws IOException { + public boolean isInECZone(INodesInPath iip) throws IOException { return getECSchema(iip) != null; } 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 47bfa47d7bc40..592c421619bf3 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 @@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; @@ -416,7 +417,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.getECPolicy(iip)); + updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip)); break; } case OP_CLOSE: { @@ -437,7 +438,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, addCloseOp, iip, file, fsDir.getECPolicy(iip)); + updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip)); // Now close the file if (!file.isUnderConstruction() && @@ -496,7 +497,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.getECPolicy(iip)); + updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip)); if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); @@ -514,7 +515,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // add the new block to the INodeFile // TODO whether the file is striped should later be retrieved from iip - addNewBlock(addBlockOp, oldFile, fsDir.getECPolicy(iip)); + addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip)); break; } case OP_SET_REPLICATION: { @@ -1079,8 +1080,9 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // 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, - file.getPreferredBlockReplication()); + newBI = isStriped ? new BlockInfoStriped(newBlock, + HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) : + new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); } fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file); file.addBlock(newBI); 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 54d0d30c24ce1..6f485f53fecf9 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 @@ -771,10 +771,8 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, if (isStriped) { blocks = new Block[numBlocks]; for (int j = 0; j < numBlocks; j++) { - short dataBlockNum = in.readShort(); - short parityBlockNum = in.readShort(); blocks[j] = new BlockInfoStriped(new Block(), - dataBlockNum, parityBlockNum); + HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); blocks[j].readFields(in); } } else { 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 58244e5bfe104..25febd45f3bad 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 @@ -32,6 +32,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.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; @@ -139,17 +140,15 @@ static INodeFile readINodeUnderConstruction( blocksStriped = new BlockInfoStriped[numBlocks]; int i = 0; for (; i < numBlocks - 1; i++) { - short dataBlockNum = in.readShort(); - short parityBlockNum = in.readShort(); - blocksStriped[i] = new BlockInfoStriped(new Block(), dataBlockNum, - parityBlockNum); + blocksStriped[i] = new BlockInfoStriped(new Block(), + HdfsConstants.NUM_DATA_BLOCKS, + HdfsConstants.NUM_PARITY_BLOCKS); blocksStriped[i].readFields(in); } if (numBlocks > 0) { - short dataBlockNum = in.readShort(); - short parityBlockNum = in.readShort(); blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(), - dataBlockNum, parityBlockNum, BlockUCState.UNDER_CONSTRUCTION, null); + HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS, + BlockUCState.UNDER_CONSTRUCTION, null); blocksStriped[i].readFields(in); } } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java index c4db5d4d37447..3b689ebd4a5a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; @@ -43,12 +42,8 @@ public class TestBlockInfoStriped { private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; private static final long BASE_ID = -1600; private static final Block baseBlock = new Block(BASE_ID); - private BlockInfoStriped info; - - @Before - public void setup() { - info = new BlockInfoStriped(baseBlock, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); - } + private final BlockInfoStriped info = new BlockInfoStriped(baseBlock, + NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); private Block[] createReportedBlocks(int num) { Block[] blocks = new Block[num]; @@ -230,17 +225,14 @@ public void testWrite() { long blkID = 1; long numBytes = 1; long generationStamp = 1; - short dataBlockNum = 6; - short parityBlockNum = 3; - ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE/Byte.SIZE*3 - + Short.SIZE/Byte.SIZE*2); - byteBuffer.putShort(dataBlockNum).putShort(parityBlockNum) - .putLong(blkID).putLong(numBytes).putLong(generationStamp); + ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE / Byte.SIZE * 3); + byteBuffer.putLong(blkID).putLong(numBytes).putLong(generationStamp); ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutput out = new DataOutputStream(byteStream); - BlockInfoStriped blk = new BlockInfoStriped(new Block(1,1,1), - (short)6,(short)3); + BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes, + generationStamp), NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + try { blk.write(out); } catch(Exception ex) { @@ -249,5 +241,4 @@ public void testWrite() { assertEquals(byteBuffer.array().length, byteStream.toByteArray().length); assertArrayEquals(byteBuffer.array(), byteStream.toByteArray()); } - } 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 c482f1faf2daf..bb37534667253 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 @@ -158,7 +158,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con for (int i = 0; i < stripedBlks.length; i++) { stripedBlks[i] = new BlockInfoStriped( new Block(stripedBlkId + i, preferredBlockSize, timestamp), - (short) 6, (short) 3); + HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); file.getStripedBlocksFeature().addBlock(stripedBlks[i]); } From a1c9425265d2c94bfc6afb39ab2c16b4ef9e874e Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Sat, 18 Apr 2015 23:20:45 +0530 Subject: [PATCH 072/212] HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for making it ready for transfer to DN (Contributed by Uma Maheswara Rao G) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/protocolPB/PBHelper.java | 137 +++++++++++++++++- .../blockmanagement/DatanodeDescriptor.java | 31 +--- .../blockmanagement/DatanodeManager.java | 4 +- .../protocol/BlockECRecoveryCommand.java | 80 +++++++++- .../server/protocol/DatanodeProtocol.java | 2 +- .../src/main/proto/DatanodeProtocol.proto | 8 + .../src/main/proto/erasurecoding.proto | 13 ++ .../hadoop/hdfs/protocolPB/TestPBHelper.java | 88 +++++++++++ .../namenode/TestRecoverStripedBlocks.java | 10 +- 10 files changed, 335 insertions(+), 41 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0ed61cd445e99..40517e79d6570 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -87,3 +87,6 @@ startup. (Hui Zheng via szetszwo) HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz). + + HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for + making it ready for transfer to DN (Uma Maheswara Rao G via vinayakumarb) 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 9ca73ae6ddca7..c127b5f5c1fdb 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 @@ -28,6 +28,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.List; @@ -100,7 +101,7 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos; +import org.apache.hadoop.hdfs.protocol.proto.*; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto; @@ -121,6 +122,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto; +import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto; @@ -132,11 +134,11 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; 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.BlockStoragePolicyProto; @@ -184,7 +186,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto; -import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos; import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto; @@ -204,8 +205,10 @@ import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand; import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; @@ -3150,4 +3153,132 @@ public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) { return new ECZoneInfo(ecZoneInfoProto.getDir(), convertECSchema(ecZoneInfoProto.getSchema())); } + + public static BlockECRecoveryInfo convertBlockECRecoveryInfo( + BlockECRecoveryInfoProto blockEcRecoveryInfoProto) { + ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock(); + ExtendedBlock block = convert(blockProto); + + DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto + .getSourceDnInfos(); + DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto); + + DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto + .getTargetDnInfos(); + DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto); + + StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto + .getTargetStorageUuids(); + String[] targetStorageUuids = convert(targetStorageUuidsProto); + + StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto + .getTargetStorageTypes(); + StorageType[] convertStorageTypes = convertStorageTypes( + targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto + .getStorageTypesList().size()); + + List liveBlockIndicesList = blockEcRecoveryInfoProto + .getLiveBlockIndicesList(); + short[] liveBlkIndices = new short[liveBlockIndicesList.size()]; + for (int i = 0; i < liveBlockIndicesList.size(); i++) { + liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue(); + } + + return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos, + targetStorageUuids, convertStorageTypes, liveBlkIndices); + } + + public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( + BlockECRecoveryInfo blockEcRecoveryInfo) { + BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto + .newBuilder(); + builder.setBlock(convert(blockEcRecoveryInfo.getExtendedBlock())); + + DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos(); + builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos)); + + DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos(); + builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos)); + + String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs(); + builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs)); + + StorageType[] targetStorageTypes = blockEcRecoveryInfo + .getTargetStorageTypes(); + builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes)); + + short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices(); + builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices)); + + return builder.build(); + } + + private static List convertIntArray(short[] liveBlockIndices) { + List liveBlockIndicesList = new ArrayList(); + for (short s : liveBlockIndices) { + liveBlockIndicesList.add((int) s); + } + return liveBlockIndicesList; + } + + private static StorageTypesProto convertStorageTypesProto( + StorageType[] targetStorageTypes) { + StorageTypesProto.Builder builder = StorageTypesProto.newBuilder(); + for (StorageType storageType : targetStorageTypes) { + builder.addStorageTypes(convertStorageType(storageType)); + } + return builder.build(); + } + + private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) { + StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder(); + for (String storageUuid : targetStorageIDs) { + builder.addStorageUuids(storageUuid); + } + return builder.build(); + } + + private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) { + DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder(); + for (DatanodeInfo datanodeInfo : dnInfos) { + builder.addDatanodes(convert(datanodeInfo)); + } + return builder.build(); + } + + private static String[] convert(StorageUuidsProto targetStorageUuidsProto) { + List storageUuidsList = targetStorageUuidsProto + .getStorageUuidsList(); + String[] storageUuids = new String[storageUuidsList.size()]; + for (int i = 0; i < storageUuidsList.size(); i++) { + storageUuids[i] = storageUuidsList.get(i); + } + return storageUuids; + } + + public static BlockECRecoveryCommandProto convert( + BlockECRecoveryCommand blkECRecoveryCmd) { + BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto + .newBuilder(); + Collection blockECRecoveryInfos = blkECRecoveryCmd + .getECTasks(); + for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) { + builder + .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo)); + } + return builder.build(); + } + + public static BlockECRecoveryCommand convert( + BlockECRecoveryCommandProto blkECRecoveryCmdProto) { + Collection blkECRecoveryInfos = new ArrayList(); + List blockECRecoveryinfoList = blkECRecoveryCmdProto + .getBlockECRecoveryinfoList(); + for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) { + blkECRecoveryInfos + .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto)); + } + return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, + blkECRecoveryInfos); + } } 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 7ec71a27e090b..35cc31b7088ec 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 @@ -32,8 +32,8 @@ import java.util.Arrays; import com.google.common.annotations.VisibleForTesting; - import com.google.common.collect.ImmutableList; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.StorageReport; @@ -99,34 +100,6 @@ public static class BlockTargetPair { } } - /** Block and targets pair */ - @InterfaceAudience.Private - @InterfaceStability.Evolving - public static class BlockECRecoveryInfo { - public final ExtendedBlock block; - public final DatanodeDescriptor[] sources; - public final DatanodeStorageInfo[] targets; - public final short[] liveBlockIndices; - - BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources, - DatanodeStorageInfo[] targets, short[] liveBlockIndices) { - this.block = block; - this.sources = sources; - this.targets = targets; - this.liveBlockIndices = liveBlockIndices; - } - - @Override - public String toString() { - return new StringBuilder().append("BlockECRecoveryInfo(\n "). - append("Recovering ").append(block). - append(" From: ").append(Arrays.asList(sources)). - append(" To: ").append(Arrays.asList(targets)).append(")\n"). - append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)). - toString(); - } - } - /** A BlockTargetPair queue. */ private static class BlockQueue { private final Queue blockq = new LinkedList(); 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 8a78a0be99136..6e84b3ed7b528 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 @@ -34,12 +34,12 @@ import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.protocol.*; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.*; @@ -1442,7 +1442,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, List pendingECList = nodeinfo.getErasureCodeCommand(maxTransfers); if (pendingECList != null) { - cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_CODEC, + cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, pendingECList)); } //check block invalidation diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java index f7f02fdea41c2..9a387dd9e0d85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java @@ -18,10 +18,15 @@ package org.apache.hadoop.hdfs.server.protocol; import com.google.common.base.Joiner; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import java.util.Arrays; import java.util.Collection; /** @@ -60,4 +65,77 @@ public String toString() { sb.append("\n)"); return sb.toString(); } + + /** Block and targets pair */ + @InterfaceAudience.Private + @InterfaceStability.Evolving + public static class BlockECRecoveryInfo { + private final ExtendedBlock block; + private final DatanodeInfo[] sources; + private DatanodeInfo[] targets; + private String[] targetStorageIDs; + private StorageType[] targetStorageTypes; + private final short[] liveBlockIndices; + + public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, + DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices) { + this.block = block; + this.sources = sources; + this.targets = DatanodeStorageInfo.toDatanodeInfos(targetDnStorageInfo); + this.targetStorageIDs = DatanodeStorageInfo + .toStorageIDs(targetDnStorageInfo); + this.targetStorageTypes = DatanodeStorageInfo + .toStorageTypes(targetDnStorageInfo); + this.liveBlockIndices = liveBlockIndices; + } + + public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, + DatanodeInfo[] targets, String[] targetStorageIDs, + StorageType[] targetStorageTypes, short[] liveBlockIndices) { + this.block = block; + this.sources = sources; + this.targets = targets; + this.targetStorageIDs = targetStorageIDs; + this.targetStorageTypes = targetStorageTypes; + this.liveBlockIndices = liveBlockIndices; + } + + public ExtendedBlock getExtendedBlock() { + return block; + } + + public DatanodeInfo[] getSourceDnInfos() { + return sources; + } + + public DatanodeInfo[] getTargetDnInfos() { + return targets; + } + + public String[] getTargetStorageIDs() { + return targetStorageIDs; + } + + public StorageType[] getTargetStorageTypes() { + return targetStorageTypes; + } + + public short[] getLiveBlockIndices() { + return liveBlockIndices; + } + + @Override + public String toString() { + return new StringBuilder().append("BlockECRecoveryInfo(\n ") + .append("Recovering ").append(block).append(" From: ") + .append(Arrays.asList(sources)).append(" To: [") + .append(Arrays.asList(targets)).append(")\n") + .append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)) + .toString(); + } + } + + public Collection getECTasks() { + return this.ecTasks; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java index b8ac165db5935..1411fa9200bb4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java @@ -76,7 +76,7 @@ public interface DatanodeProtocol { final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth final static int DNA_CACHE = 9; // cache blocks final static int DNA_UNCACHE = 10; // uncache blocks - final static int DNA_CODEC = 11; // uncache blocks + final static int DNA_ERASURE_CODING_RECOVERY = 11; // erasure coding recovery command /** * Register Datanode. 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 3083dc90f6d9c..ac9ab460bdfb3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto @@ -33,6 +33,7 @@ package hadoop.hdfs.datanode; import "HAServiceProtocol.proto"; import "hdfs.proto"; +import "erasurecoding.proto"; /** * Information to identify a datanode to a namenode @@ -144,6 +145,13 @@ message RegisterCommandProto { // void } +/** + * Block Erasure coding recovery command + */ +message BlockECRecoveryCommandProto { + repeated BlockECRecoveryInfoProto blockECRecoveryinfo = 1; +} + /** * registration - Information of the datanode registering with the namenode */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index d888f71ef9842..59bd9497ef0cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -21,6 +21,7 @@ option java_outer_classname = "ErasureCodingProtos"; option java_generate_equals_and_hash = true; package hadoop.hdfs; +import "hdfs.proto"; /** * ECSchema options entry @@ -86,4 +87,16 @@ message GetECZoneInfoRequestProto { message GetECZoneInfoResponseProto { optional ECZoneInfoProto ECZoneInfo = 1; +} + +/** + * Block erasure coding recovery info + */ +message BlockECRecoveryInfoProto { + required ExtendedBlockProto block = 1; + required DatanodeInfosProto sourceDnInfos = 2; + required DatanodeInfosProto targetDnInfos = 3; + required StorageUuidsProto targetStorageUuids = 4; + required StorageTypesProto targetStorageTypes = 5; + repeated uint32 liveBlockIndices = 6; } \ No newline at end of file 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 4b42f4cd3e977..4ec4ea5214f0b 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 @@ -24,6 +24,8 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; import java.util.List; import org.apache.hadoop.fs.permission.AclEntry; @@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto; +import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto; 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.HdfsProtos; @@ -63,15 +66,20 @@ import org.apache.hadoop.hdfs.security.token.block.BlockKey; 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.BlockManagerTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.StorageInfo; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; 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.BlockECRecoveryCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; @@ -639,4 +647,84 @@ public void testAclStatusProto() { .build(); Assert.assertEquals(s, PBHelper.convert(PBHelper.convert(s))); } + + @Test + public void testBlockECRecoveryCommand() { + DatanodeInfo[] dnInfos0 = new DatanodeInfo[] { + DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() }; + DatanodeStorageInfo targetDnInfos_0 = BlockManagerTestUtil + .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(), + new DatanodeStorage("s00")); + DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil + .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(), + new DatanodeStorage("s01")); + DatanodeStorageInfo[] targetDnInfos0 = new DatanodeStorageInfo[] { + targetDnInfos_0, targetDnInfos_1 }; + short[] liveBlkIndices0 = new short[2]; + BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo( + new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0, + liveBlkIndices0); + DatanodeInfo[] dnInfos1 = new DatanodeInfo[] { + DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() }; + DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil + .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(), + new DatanodeStorage("s02")); + DatanodeStorageInfo targetDnInfos_3 = BlockManagerTestUtil + .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(), + new DatanodeStorage("s03")); + DatanodeStorageInfo[] targetDnInfos1 = new DatanodeStorageInfo[] { + targetDnInfos_2, targetDnInfos_3 }; + short[] liveBlkIndices1 = new short[2]; + BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo( + new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1, + liveBlkIndices1); + List blkRecoveryInfosList = new ArrayList(); + blkRecoveryInfosList.add(blkECRecoveryInfo0); + blkRecoveryInfosList.add(blkECRecoveryInfo1); + BlockECRecoveryCommand blkECRecoveryCmd = new BlockECRecoveryCommand( + DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, blkRecoveryInfosList); + BlockECRecoveryCommandProto blkECRecoveryCmdProto = PBHelper + .convert(blkECRecoveryCmd); + blkECRecoveryCmd = PBHelper.convert(blkECRecoveryCmdProto); + Iterator iterator = blkECRecoveryCmd.getECTasks() + .iterator(); + assertBlockECRecoveryInfoEquals(blkECRecoveryInfo0, iterator.next()); + assertBlockECRecoveryInfoEquals(blkECRecoveryInfo1, iterator.next()); + } + + private void assertBlockECRecoveryInfoEquals( + BlockECRecoveryInfo blkECRecoveryInfo1, + BlockECRecoveryInfo blkECRecoveryInfo2) { + assertEquals(blkECRecoveryInfo1.getExtendedBlock(), + blkECRecoveryInfo2.getExtendedBlock()); + + DatanodeInfo[] sourceDnInfos1 = blkECRecoveryInfo1.getSourceDnInfos(); + DatanodeInfo[] sourceDnInfos2 = blkECRecoveryInfo2.getSourceDnInfos(); + assertDnInfosEqual(sourceDnInfos1, sourceDnInfos2); + + DatanodeInfo[] targetDnInfos1 = blkECRecoveryInfo1.getTargetDnInfos(); + DatanodeInfo[] targetDnInfos2 = blkECRecoveryInfo2.getTargetDnInfos(); + assertDnInfosEqual(targetDnInfos1, targetDnInfos2); + + String[] targetStorageIDs1 = blkECRecoveryInfo1.getTargetStorageIDs(); + String[] targetStorageIDs2 = blkECRecoveryInfo2.getTargetStorageIDs(); + assertEquals(targetStorageIDs1.length, targetStorageIDs2.length); + for (int i = 0; i < targetStorageIDs1.length; i++) { + assertEquals(targetStorageIDs1[i], targetStorageIDs2[i]); + } + + short[] liveBlockIndices1 = blkECRecoveryInfo1.getLiveBlockIndices(); + short[] liveBlockIndices2 = blkECRecoveryInfo2.getLiveBlockIndices(); + for (int i = 0; i < liveBlockIndices1.length; i++) { + assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]); + } + } + + private void assertDnInfosEqual(DatanodeInfo[] dnInfos1, + DatanodeInfo[] dnInfos2) { + assertEquals(dnInfos1.length, dnInfos2.length); + for (int i = 0; i < dnInfos1.length; i++) { + compare(dnInfos1[i], dnInfos2[i]); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index ea18c3ee66151..ca4fbbc99cc54 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -29,9 +29,9 @@ 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.DatanodeDescriptor.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -115,10 +115,10 @@ public void testMissingStripedBlock() throws Exception { last.getNumberOfBlocksToBeErasureCoded()); List recovery = last.getErasureCodeCommand(numBlocks); for (BlockECRecoveryInfo info : recovery) { - assertEquals(1, info.targets.length); - assertEquals(last, info.targets[0].getDatanodeDescriptor()); - assertEquals(GROUP_SIZE - 1, info.sources.length); - assertEquals(GROUP_SIZE - 1, info.liveBlockIndices.length); + assertEquals(1, info.getTargetDnInfos().length); + assertEquals(last, info.getTargetDnInfos()[0]); + assertEquals(GROUP_SIZE - 1, info.getSourceDnInfos().length); + assertEquals(GROUP_SIZE - 1, info.getLiveBlockIndices().length); } } } From dfba46ab577ef76b681d1848ded2be269079602a Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 20 Apr 2015 15:04:49 +0530 Subject: [PATCH 073/212] HDFS-8181. createErasureCodingZone sets retryCache state as false always (Contributed by Uma Maheswara Rao G) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 14 ++++++++++++++ .../hdfs/server/namenode/NameNodeRpcServer.java | 1 + 2 files changed, 15 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 40517e79d6570..c8dbf083fc054 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -83,10 +83,24 @@ HDFS-7349. Support DFS command for the EC encoding (vinayakumarb) + HDFS-8120. Erasure coding: created util class to analyze striped block groups. + (Contributed by Zhe Zhang and Li Bo via Jing Zhao) + HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. (Hui Zheng via szetszwo) HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz). + HDFS-8166. DFSStripedOutputStream should not create empty blocks. (Jing Zhao) + + HDFS-7937. Erasure Coding: INodeFile quota computation unit tests. + (Kai Sasaki via Jing Zhao) + + HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks. + (Jing Zhao) + HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for making it ready for transfer to DN (Uma Maheswara Rao G via vinayakumarb) + + HDFS-8181. createErasureCodingZone sets retryCache state as false always + (Uma Maheswara Rao G via vinayakumarb) 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 8217907cb7947..dcf06070ace5a 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 @@ -1834,6 +1834,7 @@ public void createErasureCodingZone(String src, ECSchema schema) boolean success = false; try { namesystem.createErasureCodingZone(src, schema, cacheEntry != null); + success = true; } finally { RetryCache.setState(cacheEntry, success); } From 922631f04f4bc79f4ceb17958030f80beb4f0576 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 20 Apr 2015 14:19:12 -0700 Subject: [PATCH 074/212] HDFS-8188. Erasure coding: refactor client-related code to sync with HDFS-8082 and HDFS-8169. Contributed by Zhe Zhang. --- .../hdfs/client/HdfsClientConfigKeys.java | 12 +++++++++ .../hdfs/protocol/LocatedStripedBlock.java | 9 ------- .../org/apache/hadoop/hdfs/DFSClient.java | 21 +++++----------- .../hdfs/client/impl/DfsClientConf.java | 21 ++++++++++++++-- .../server/blockmanagement/BlockManager.java | 25 ++++++++++++------- .../server/namenode/TestStripedINodeFile.java | 3 ++- 6 files changed, 55 insertions(+), 36 deletions(-) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java (84%) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 26283aade86c0..6006d71faf6a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -177,6 +177,18 @@ interface HedgedRead { int THREADPOOL_SIZE_DEFAULT = 0; } + /** dfs.client.read.striped configuration properties */ + interface StripedRead { + String PREFIX = Read.PREFIX + "striped."; + + String THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size"; + /** + * With default 6+3 schema, each normal read could span 6 DNs. So this + * default value accommodates 3 read streams + */ + int THREADPOOL_SIZE_DEFAULT = 18; + } + /** dfs.http.client configuration properties */ interface HttpClient { String PREFIX = "dfs.http.client."; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java similarity index 84% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java index 98614db446538..93a5948007cdc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -20,7 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; -import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import java.util.Arrays; @@ -43,14 +42,6 @@ public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs, System.arraycopy(indices, 0, blockIndices, 0, indices.length); } - public LocatedStripedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages, - int[] indices, long startOffset, boolean corrupt) { - this(b, DatanodeStorageInfo.toDatanodeInfos(storages), - DatanodeStorageInfo.toStorageIDs(storages), - DatanodeStorageInfo.toStorageTypes(storages), indices, - startOffset, corrupt, EMPTY_LOCS); - } - @Override public String toString() { return getClass().getSimpleName() + "{" + getBlock() diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 9c17a4ef17703..0a67cf7550e05 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -382,21 +382,12 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, dfsClientConf); if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) { - this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize()); - } - numThreads = conf.getInt( - DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE, - DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE); - if (numThreads <= 0) { - LOG.warn("The value of " - + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE - + " must be greater than 0. The current setting is " + numThreads - + ". Reset it to the default value " - + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE); - numThreads = - DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE; - } - this.initThreadsNumForStripedReads(numThreads); + this.initThreadsNumForHedgedReads(dfsClientConf. + getHedgedReadThreadpoolSize()); + } + + this.initThreadsNumForStripedReads(dfsClientConf. + getStripedReadThreadpoolSize()); this.saslClient = new SaslDataTransferClient( conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf), TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java index a257e32bed0df..32a3da0a7c28d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java @@ -38,6 +38,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; +import com.google.common.base.Preconditions; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.HadoopIllegalArgumentException; @@ -101,6 +102,8 @@ public class DfsClientConf { private final long hedgedReadThresholdMillis; private final int hedgedReadThreadpoolSize; + private final int stripedReadThreadpoolSize; + public DfsClientConf(Configuration conf) { // The hdfsTimeout is currently the same as the ipc timeout hdfsTimeout = Client.getTimeout(conf); @@ -191,7 +194,7 @@ public DfsClientConf(Configuration conf) { connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME, DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT); hdfsBlocksMetadataEnabled = conf.getBoolean( - DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, + DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT); fileBlockStorageLocationsNumThreads = conf.getInt( DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS, @@ -215,6 +218,13 @@ public DfsClientConf(Configuration conf) { hedgedReadThreadpoolSize = conf.getInt( HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_DEFAULT); + + stripedReadThreadpoolSize = conf.getInt( + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY, + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_DEFAULT); + Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " + + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY + + " must be greater than 0."); } private DataChecksum.Type getChecksumType(Configuration conf) { @@ -491,6 +501,13 @@ public int getHedgedReadThreadpoolSize() { return hedgedReadThreadpoolSize; } + /** + * @return the stripedReadThreadpoolSize + */ + public int getStripedReadThreadpoolSize() { + return stripedReadThreadpoolSize; + } + /** * @return the shortCircuitConf */ @@ -744,4 +761,4 @@ public String confAsString() { return builder.toString(); } } -} \ No newline at end of file +} 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 90507e99bcf69..6657e5b369491 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 @@ -874,7 +874,7 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) { final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return new LocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, + return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, false); } else { assert blk instanceof BlockInfoContiguousUnderConstruction; @@ -883,13 +883,8 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) { final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return new LocatedBlock(eb, storages, pos, false); + return newLocatedBlock(eb, storages, pos, false); } - final BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction) blk; - final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); - final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return newLocatedBlock(eb, storages, pos, false); } // get block locations @@ -932,7 +927,7 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) { final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); return blockIndices == null ? newLocatedBlock(eb, machines, pos, isCorrupt) : - new LocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt); + newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt); } /** Create a LocatedBlocks. */ @@ -3514,7 +3509,7 @@ boolean isNodeHealthyForDecommission(DatanodeDescriptor node) { if (pendingReplicationBlocksCount == 0 && underReplicatedBlocksCount == 0) { LOG.info("Node {} is dead and there are no under-replicated" + - " blocks or blocks pending replication. Safe to decommission.", + " blocks or blocks pending replication. Safe to decommission.", node); return true; } @@ -3920,6 +3915,18 @@ public static LocatedBlock newLocatedBlock( null); } + public static LocatedStripedBlock newLocatedStripedBlock( + ExtendedBlock b, DatanodeStorageInfo[] storages, + int[] indices, long startOffset, boolean corrupt) { + // startOffset is unknown + return new LocatedStripedBlock( + b, DatanodeStorageInfo.toDatanodeInfos(storages), + DatanodeStorageInfo.toStorageIDs(storages), + DatanodeStorageInfo.toStorageTypes(storages), + indices, startOffset, corrupt, + null); + } + /** * This class is used internally by {@link this#computeRecoveryWorkForBlocks} * to represent a task to recover a block through replication or erasure diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index d251c30ad86cd..4a6d6cc80d87a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; @@ -45,7 +46,7 @@ public class TestStripedINodeFile { "userName", null, FsPermission.getDefault()); private static INodeFile createStripedINodeFile() { - return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, + return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID); } From e107886d6f621b7992f2d9a74cb1c0f987c9d774 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 20 Apr 2015 17:42:02 -0700 Subject: [PATCH 075/212] HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/util/StripedBlockUtil.java | 61 +++--- .../hdfs/TestDFSStripedOutputStream.java | 178 +++++++----------- 3 files changed, 100 insertions(+), 142 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index c8dbf083fc054..8f28285a4874d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -104,3 +104,6 @@ HDFS-8181. createErasureCodingZone sets retryCache state as false always (Uma Maheswara Rao G via vinayakumarb) + + HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error. + (szetszwo) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 23680216b8dc3..d622d4dab0766 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -25,6 +25,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import com.google.common.base.Preconditions; + /** * Utility class for analyzing striped block groups */ @@ -81,46 +83,43 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, /** * Get the size of an internal block at the given index of a block group * - * @param numBytesInGroup Size of the block group only counting data blocks + * @param dataSize Size of the block group only counting data blocks * @param cellSize The size of a striping cell - * @param dataBlkNum The number of data blocks - * @param idxInGroup The logical index in the striped block group + * @param numDataBlocks The number of data blocks + * @param i The logical index in the striped block group * @return The size of the internal block at the specified index */ - public static long getInternalBlockLength(long numBytesInGroup, - int cellSize, int dataBlkNum, int idxInGroup) { + public static long getInternalBlockLength(long dataSize, + int cellSize, int numDataBlocks, int i) { + Preconditions.checkArgument(dataSize >= 0); + Preconditions.checkArgument(cellSize > 0); + Preconditions.checkArgument(numDataBlocks > 0); + Preconditions.checkArgument(i >= 0); // Size of each stripe (only counting data blocks) - final long numBytesPerStripe = cellSize * dataBlkNum; - assert numBytesPerStripe > 0: - "getInternalBlockLength should only be called on valid striped blocks"; + final int stripeSize = cellSize * numDataBlocks; // If block group ends at stripe boundary, each internal block has an equal // share of the group - if (numBytesInGroup % numBytesPerStripe == 0) { - return numBytesInGroup / dataBlkNum; + final int lastStripeDataLen = (int)(dataSize % stripeSize); + if (lastStripeDataLen == 0) { + return dataSize / numDataBlocks; } - int numStripes = (int) ((numBytesInGroup - 1) / numBytesPerStripe + 1); - assert numStripes >= 1 : "There should be at least 1 stripe"; - - // All stripes but the last one are full stripes. The block should at least - // contain (numStripes - 1) full cells. - long blkSize = (numStripes - 1) * cellSize; - - long lastStripeLen = numBytesInGroup % numBytesPerStripe; - // Size of parity cells should equal the size of the first cell, if it - // is not full. - long lastParityCellLen = Math.min(cellSize, lastStripeLen); - - if (idxInGroup >= dataBlkNum) { - // for parity blocks - blkSize += lastParityCellLen; - } else { - // for data blocks - blkSize += Math.min(cellSize, - Math.max(0, lastStripeLen - cellSize * idxInGroup)); + final int numStripes = (int) ((dataSize - 1) / stripeSize + 1); + return (numStripes - 1L)*cellSize + + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i); + } + + private static int lastCellSize(int size, int cellSize, int numDataBlocks, + int i) { + if (i < numDataBlocks) { + // parity block size (i.e. i >= numDataBlocks) is the same as + // the first data block size (i.e. i = 0). + size -= i*cellSize; + if (size < 0) { + size = 0; + } } - - return blkSize; + return size > cellSize? cellSize: size; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index cc20f4055b3d4..160b190cbe03d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -1,7 +1,31 @@ +/** + * 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 java.nio.ByteBuffer; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -17,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; - import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.IOUtils; @@ -30,13 +53,8 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.util.ArrayList; -import java.util.List; - public class TestDFSStripedOutputStream { + public static final Log LOG = LogFactory.getLog(TestDFSStripedOutputStream.class); private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; @@ -46,7 +64,6 @@ public class TestDFSStripedOutputStream { private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final int stripesPerBlock = 4; private final int blockSize = cellSize * stripesPerBlock; - private final RawErasureEncoder encoder = new RSRawEncoder(); @Before public void setup() throws IOException { @@ -56,7 +73,6 @@ public void setup() throws IOException { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); - encoder.initialize(dataBlocks, parityBlocks, cellSize); } @After @@ -67,78 +83,74 @@ public void tearDown() { } @Test - public void TestFileEmpty() throws IOException { + public void testFileEmpty() throws IOException { testOneFile("/EmptyFile", 0); } @Test - public void TestFileSmallerThanOneCell1() throws IOException { + public void testFileSmallerThanOneCell1() throws IOException { testOneFile("/SmallerThanOneCell", 1); } @Test - public void TestFileSmallerThanOneCell2() throws IOException { + public void testFileSmallerThanOneCell2() throws IOException { testOneFile("/SmallerThanOneCell", cellSize - 1); } @Test - public void TestFileEqualsWithOneCell() throws IOException { + public void testFileEqualsWithOneCell() throws IOException { testOneFile("/EqualsWithOneCell", cellSize); } @Test - public void TestFileSmallerThanOneStripe1() throws IOException { + public void testFileSmallerThanOneStripe1() throws IOException { testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1); } @Test - public void TestFileSmallerThanOneStripe2() throws IOException { + public void testFileSmallerThanOneStripe2() throws IOException { testOneFile("/SmallerThanOneStripe", cellSize + 123); } @Test - public void TestFileEqualsWithOneStripe() throws IOException { + public void testFileEqualsWithOneStripe() throws IOException { testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks); } @Test - public void TestFileMoreThanOneStripe1() throws IOException { + public void testFileMoreThanOneStripe1() throws IOException { testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123); } @Test - public void TestFileMoreThanOneStripe2() throws IOException { + public void testFileMoreThanOneStripe2() throws IOException { testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks + cellSize * dataBlocks + 123); } @Test - public void TestFileFullBlockGroup() throws IOException { + public void testFileFullBlockGroup() throws IOException { testOneFile("/FullBlockGroup", blockSize * dataBlocks); } @Test - public void TestFileMoreThanABlockGroup1() throws IOException { + public void testFileMoreThanABlockGroup1() throws IOException { testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); } @Test - public void TestFileMoreThanABlockGroup2() throws IOException { + public void testFileMoreThanABlockGroup2() throws IOException { testOneFile("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123); } @Test - public void TestFileMoreThanABlockGroup3() throws IOException { + public void testFileMoreThanABlockGroup3() throws IOException { testOneFile("/MoreThanABlockGroup3", blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123); } - private int stripeDataSize() { - return cellSize * dataBlocks; - } - private byte[] generateBytes(int cnt) { byte[] bytes = new byte[cnt]; for (int i = 0; i < cnt; i++) { @@ -152,8 +164,7 @@ private byte getByte(long pos) { return (byte) (pos % mod + 1); } - private void testOneFile(String src, int writeBytes) - throws IOException { + private void testOneFile(String src, int writeBytes) throws IOException { Path testPath = new Path(src); byte[] bytes = generateBytes(writeBytes); @@ -161,8 +172,7 @@ private void testOneFile(String src, int writeBytes) // check file length FileStatus status = fs.getFileStatus(testPath); - long fileLength = status.getLen(); - Assert.assertEquals(writeBytes, fileLength); + Assert.assertEquals(writeBytes, status.getLen()); List> blockGroupList = new ArrayList<>(); LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L); @@ -251,16 +261,12 @@ public Peer newConnectedPeer(InetSocketAddress addr, continue; } for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) { - byte expected; // calculate the position of this byte in the file long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize, dataBlocks, posInBlk, blkIdxInGroup) + group * blockSize * dataBlocks; - if (posInFile >= writeBytes) { - expected = 0; - } else { - expected = getByte(posInFile); - } + Assert.assertTrue(posInFile < writeBytes); + final byte expected = getByte(posInFile); String s = "Unexpected byte " + actualBlkBytes[posInBlk] + ", expect " + expected @@ -272,84 +278,34 @@ public Peer newConnectedPeer(InetSocketAddress addr, } } - // verify the parity blocks - final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks]; - final long groupSize = lbs.getLocatedBlocks().get(group).getBlockSize(); - int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(groupSize, - cellSize, dataBlocks, dataBlocks); - for (int i = 0; i < parityBlocks; i++) { - parityBuffers[i] = ByteBuffer.allocate(parityBlkSize); - } - final int numStripes = (int) (groupSize - 1) / stripeDataSize() + 1; - for (int i = 0; i < numStripes; i++) { - final int parityCellSize = i < numStripes - 1 || parityBlkSize % cellSize == 0 - ? cellSize : parityBlkSize % cellSize; - ByteBuffer[] stripeBuf = new ByteBuffer[dataBlocks]; - for (int k = 0; k < stripeBuf.length; k++) { - stripeBuf[k] = ByteBuffer.allocate(cellSize); - } - for (int j = 0; j < dataBlocks; j++) { - if (dataBlockBytes[j] != null) { - int length = Math.min(cellSize, - dataBlockBytes[j].length - cellSize * i); - if (length > 0) { - stripeBuf[j].put(dataBlockBytes[j], cellSize * i, length); - } - } - final long pos = stripeBuf[j].position(); - for (int k = 0; k < parityCellSize - pos; k++) { - stripeBuf[j].put((byte) 0); - } - stripeBuf[j].flip(); - } - ByteBuffer[] parityBuf = new ByteBuffer[parityBlocks]; - for (int j = 0; j < parityBlocks; j++) { - parityBuf[j] = ByteBuffer.allocate(cellSize); - for (int k = 0; k < parityCellSize; k++) { - parityBuf[j].put((byte) 0); - } - parityBuf[j].flip(); - } - - encoder.encode(stripeBuf, parityBuf); - for (int j = 0; j < parityBlocks; j++) { - parityBuffers[j].put(parityBuf[j]); - } - } - - for (int i = 0; i < parityBlocks; i++) { - Assert.assertArrayEquals(parityBuffers[i].array(), parityBlockBytes[i]); - } + verifyParity(lbs.getLocatedBlocks().get(group).getBlockSize(), + cellSize, dataBlockBytes, parityBlockBytes); } } - - private void testReadWriteOneFile(String src, int writeBytes) - throws IOException { - Path TestPath = new Path(src); - byte[] bytes = generateBytes(writeBytes); - DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); - - //check file length - FileStatus status = fs.getFileStatus(TestPath); - long fileLength = status.getLen(); - if (fileLength != writeBytes) { - Assert.fail("File Length error: expect=" + writeBytes - + ", actual=" + fileLength); + + static void verifyParity(final long size, final int cellSize, + byte[][] dataBytes, byte[][] parityBytes) { + // verify the parity blocks + int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength( + size, cellSize, dataBytes.length, dataBytes.length); + final byte[][] expectedParityBytes = new byte[parityBytes.length][]; + for (int i = 0; i < parityBytes.length; i++) { + expectedParityBytes[i] = new byte[parityBlkSize]; } - - DFSStripedInputStream dis = new DFSStripedInputStream( - fs.getClient(), src, true); - byte[] buf = new byte[writeBytes + 100]; - int readLen = dis.read(0, buf, 0, buf.length); - readLen = readLen >= 0 ? readLen : 0; - if (readLen != writeBytes) { - Assert.fail("The length of file is not correct."); - } - - for (int i = 0; i < writeBytes; i++) { - if (getByte(i) != buf[i]) { - Assert.fail("Byte at i = " + i + " is wrongly written."); + for (int i = 0; i < dataBytes.length; i++) { + if (dataBytes[i] == null) { + dataBytes[i] = new byte[dataBytes[0].length]; + } else if (dataBytes[i].length < dataBytes[0].length) { + final byte[] tmp = dataBytes[i]; + dataBytes[i] = new byte[dataBytes[0].length]; + System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length); } } + final RawErasureEncoder encoder = new RSRawEncoder(); + encoder.initialize(dataBytes.length, parityBytes.length, cellSize); + encoder.encode(dataBytes, expectedParityBytes); + for (int i = 0; i < parityBytes.length; i++) { + Assert.assertArrayEquals(expectedParityBytes[i], parityBytes[i]); + } } } From e9d85bbf304bcf070fe5f1dee3398fa1c80a50a9 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 21 Apr 2015 20:56:39 -0700 Subject: [PATCH 076/212] HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader. Contributed by Tsz Wo Nicholas Sze. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/BlockReaderTestUtil.java | 7 +-- .../hadoop/hdfs/TestBlockReaderFactory.java | 16 ++--- .../hdfs/TestDFSStripedOutputStream.java | 58 ++----------------- 4 files changed, 20 insertions(+), 64 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8f28285a4874d..d8f2e9dbcc047 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -107,3 +107,6 @@ HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error. (szetszwo) + + HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to + create BlockReader. (szetszwo via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java index 88b7f37dcced3..829cf03c86b88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java @@ -165,20 +165,19 @@ public void readAndCheckEOS(BlockReader reader, int length, boolean expectEof) */ public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead) throws IOException { - return getBlockReader(cluster, testBlock, offset, lenToRead); + return getBlockReader(cluster.getFileSystem(), testBlock, offset, lenToRead); } /** * Get a BlockReader for the given block. */ - public static BlockReader getBlockReader(MiniDFSCluster cluster, - LocatedBlock testBlock, int offset, int lenToRead) throws IOException { + public static BlockReader getBlockReader(final DistributedFileSystem fs, + LocatedBlock testBlock, int offset, long lenToRead) throws IOException { InetSocketAddress targetAddr = null; ExtendedBlock block = testBlock.getBlock(); DatanodeInfo[] nodes = testBlock.getLocations(); targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); - final DistributedFileSystem fs = cluster.getFileSystem(); return new BlockReaderFactory(fs.getClient().getConf()). setInetSocketAddress(targetAddr). setBlock(block). diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java index d8acefff24eef..1a767c320e38e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java @@ -250,8 +250,8 @@ public void run() { LocatedBlock lblock = locatedBlocks.get(0); // first block BlockReader blockReader = null; try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); Assert.fail("expected getBlockReader to fail the first time."); } catch (Throwable t) { Assert.assertTrue("expected to see 'TCP reads were disabled " + @@ -265,8 +265,8 @@ public void run() { // Second time should succeed. try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); } catch (Throwable t) { LOG.error("error trying to retrieve a block reader " + "the second time.", t); @@ -474,8 +474,8 @@ public void run() { while (true) { BlockReader blockReader = null; try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); sem.release(); try { blockReader.readAll(buf, 0, TEST_FILE_LEN); @@ -514,8 +514,8 @@ public void run() { // getting a ClosedChannelException. BlockReader blockReader = null; try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); blockReader.readFully(buf, 0, TEST_FILE_LEN); } finally { if (blockReader != null) blockReader.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 160b190cbe03d..c2131834b8122 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hdfs; import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.Socket; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -29,25 +27,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.client.impl.DfsClientConf; -import org.apache.hadoop.hdfs.net.Peer; -import org.apache.hadoop.hdfs.net.TcpPeerServer; -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.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; -import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; -import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.token.Token; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -59,7 +46,6 @@ public class TestDFSStripedOutputStream { private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; private MiniDFSCluster cluster; - private Configuration conf = new Configuration(); private DistributedFileSystem fs; private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final int stripesPerBlock = 4; @@ -173,7 +159,11 @@ private void testOneFile(String src, int writeBytes) throws IOException { // check file length FileStatus status = fs.getFileStatus(testPath); Assert.assertEquals(writeBytes, status.getLen()); + + checkData(src, writeBytes); + } + void checkData(String src, int writeBytes) throws IOException { List> blockGroupList = new ArrayList<>(); LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L); @@ -199,11 +189,7 @@ private void testOneFile(String src, int writeBytes) throws IOException { if (lblock == null) { continue; } - DatanodeInfo[] nodes = lblock.getLocations(); ExtendedBlock block = lblock.getBlock(); - InetSocketAddress targetAddr = NetUtils.createSocketAddr( - nodes[0].getXferAddr()); - byte[] blockBytes = new byte[(int)block.getNumBytes()]; if (i < dataBlocks) { dataBlockBytes[i] = blockBytes; @@ -215,40 +201,8 @@ private void testOneFile(String src, int writeBytes) throws IOException { continue; } - BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)). - setFileName(src). - setBlock(block). - setBlockToken(lblock.getBlockToken()). - setInetSocketAddress(targetAddr). - setStartOffset(0). - setLength(block.getNumBytes()). - setVerifyChecksum(true). - setClientName("TestStripeLayoutWrite"). - setDatanodeInfo(nodes[0]). - setCachingStrategy(CachingStrategy.newDefaultStrategy()). - setClientCacheContext(ClientContext.getFromConf(conf)). - setConfiguration(conf). - setRemotePeerFactory(new RemotePeerFactory() { - @Override - public Peer newConnectedPeer(InetSocketAddress addr, - Token blockToken, - DatanodeID datanodeId) - throws IOException { - Peer peer = null; - Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); - try { - sock.connect(addr, HdfsServerConstants.READ_TIMEOUT); - sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); - peer = TcpPeerServer.peerFromSocket(sock); - } finally { - if (peer == null) { - IOUtils.closeSocket(sock); - } - } - return peer; - } - }).build(); - + final BlockReader blockReader = BlockReaderTestUtil.getBlockReader( + fs, lblock, 0, block.getNumBytes()); blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); blockReader.close(); } From 2f11109bb0db1f3baf802a345f2de840b1553493 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 21 Apr 2015 21:03:07 -0700 Subject: [PATCH 077/212] HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver. Contributed by Tsz Wo Nicholas Sze. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DistributedFileSystem.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index d8f2e9dbcc047..3d86f05336e74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -110,3 +110,6 @@ HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader. (szetszwo via Zhe Zhang) + + HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema + in FileSystemLinkResolver. (szetszwo via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index d4749f4ba209b..d471271295db6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -2290,7 +2290,7 @@ public void createErasureCodingZone(final Path path, final ECSchema schema) @Override public Void doCall(final Path p) throws IOException, UnresolvedLinkException { - dfs.createErasureCodingZone(getPathName(p), null); + dfs.createErasureCodingZone(getPathName(p), schema); return null; } From 014d8675c59d44ad68dec36db6afe3f3666a3f15 Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Wed, 22 Apr 2015 19:30:14 +0530 Subject: [PATCH 078/212] HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. (Contributed by Uma Maheswara Rao G) --- .../coder/AbstractErasureCoder.java | 2 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/protocolPB/PBHelper.java | 7 ++ .../hdfs/server/datanode/BPOfferService.java | 6 ++ .../hadoop/hdfs/server/datanode/DataNode.java | 10 +++ .../erasurecode/ErasureCodingWorker.java | 83 +++++++++++++++++++ .../src/main/proto/DatanodeProtocol.proto | 2 + 7 files changed, 112 insertions(+), 1 deletion(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index e5bf11a2ac67e..7403e35c946ff 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -66,7 +66,7 @@ protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) { * @param isEncoder * @return raw coder */ - protected static RawErasureCoder createRawCoder(Configuration conf, + public static RawErasureCoder createRawCoder(Configuration conf, String rawCoderFactoryKey, boolean isEncoder) { if (conf == null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 3d86f05336e74..1acde41811726 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -113,3 +113,6 @@ HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver. (szetszwo via Zhe Zhang) + + HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. + (umamahesh) 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 c127b5f5c1fdb..68cfe7f57d49b 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 @@ -973,6 +973,8 @@ public static DatanodeCommand convert(DatanodeCommandProto proto) { return REG_CMD; case BlockIdCommand: return PBHelper.convert(proto.getBlkIdCmd()); + case BlockECRecoveryCommand: + return PBHelper.convert(proto.getBlkECRecoveryCmd()); default: return null; } @@ -1123,6 +1125,11 @@ public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) { builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand). setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand)); break; + case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY: + builder.setCmdType(DatanodeCommandProto.Type.BlockECRecoveryCommand) + .setBlkECRecoveryCmd( + convert((BlockECRecoveryCommand) datanodeCommand)); + break; case DatanodeProtocol.DNA_UNKNOWN: //Not expected default: builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand); 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 92323f1530b9f..69baac7ec3681 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 @@ -32,11 +32,13 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.server.protocol.*; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; @@ -722,6 +724,10 @@ assert getBlockPoolId().equals(bp) : dxcs.balanceThrottler.setBandwidth(bandwidth); } break; + case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY: + LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY"); + Collection ecTasks = ((BlockECRecoveryCommand) cmd).getECTasks(); + dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks); default: LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction()); } 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 d2b293957df9b..221ba386ac190 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 @@ -87,6 +87,7 @@ import javax.management.ObjectName; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -152,6 +153,7 @@ import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources; +import org.apache.hadoop.hdfs.server.datanode.erasurecode.ErasureCodingWorker; 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.metrics.DataNodeMetrics; @@ -354,6 +356,8 @@ public static InetSocketAddress createSocketAddr(String target) { private String dnUserName = null; private SpanReceiverHost spanReceiverHost; + + private ErasureCodingWorker ecWorker; private static final int NUM_CORES = Runtime.getRuntime() .availableProcessors(); private static final double CONGESTION_RATIO = 1.5; @@ -1157,6 +1161,7 @@ void startDataNode(Configuration conf, saslClient = new SaslDataTransferClient(dnConf.conf, dnConf.saslPropsResolver, dnConf.trustedChannelResolver); saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager); + ecWorker = new ErasureCodingWorker(conf); // Initialize ErasureCoding worker } /** @@ -3258,4 +3263,9 @@ public void removeSpanReceiver(long id) throws IOException { checkSuperuserPrivilege(); spanReceiverHost.removeSpanReceiver(id); } + + public ErasureCodingWorker getErasureCodingWorker(){ + return ecWorker; + + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java new file mode 100644 index 0000000000000..6430308e7d5e0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -0,0 +1,83 @@ +/** + * 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.erasurecode; + +import java.util.Collection; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; +import org.apache.hadoop.io.erasurecode.coder.AbstractErasureCoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder; + +/** + * ErasureCodingWorker handles the erasure coding recovery work commands. These + * commands would be issued from Namenode as part of Datanode's heart beat + * response. BPOfferService delegates the work to this class for handling EC + * commands. + */ +public final class ErasureCodingWorker { + + private Configuration conf; + RawErasureCoder rawEncoder = null; + RawErasureCoder rawDecoder = null; + + public ErasureCodingWorker(Configuration conf) { + this.conf = conf; + initialize(); + } + + /** + * Initializes the required resources for handling the erasure coding recovery + * work. + */ + public void initialize() { + // Right now directly used RS coder. Once other coders integration ready, we + // can load preferred codec here. + initializeErasureEncoder(); + initializeErasureDecoder(); + } + + private void initializeErasureDecoder() { + rawDecoder = AbstractErasureCoder.createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, false); + if (rawDecoder == null) { + rawDecoder = new RSRawDecoder(); + } + } + + private void initializeErasureEncoder() { + rawEncoder = AbstractErasureCoder.createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, true); + if (rawEncoder == null) { + rawEncoder = new RSRawEncoder(); + } + } + + /** + * Handles the Erasure Coding recovery work commands. + * + * @param ecTasks + * BlockECRecoveryInfo + */ + public void processErasureCodingTasks(Collection ecTasks) { + // HDFS-7348 : Implement the actual recovery process + } +} 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 ac9ab460bdfb3..482e2e9b7bcba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto @@ -59,6 +59,7 @@ message DatanodeCommandProto { UnusedUpgradeCommand = 6; NullDatanodeCommand = 7; BlockIdCommand = 8; + BlockECRecoveryCommand = 9; } required Type cmdType = 1; // Type of the command @@ -72,6 +73,7 @@ message DatanodeCommandProto { optional KeyUpdateCommandProto keyUpdateCmd = 6; optional RegisterCommandProto registerCmd = 7; optional BlockIdCommandProto blkIdCmd = 8; + optional BlockECRecoveryCommandProto blkECRecoveryCmd = 9; } /** From e8df2581c3293d0b6c43862edbf034f9e7851dbf Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 22 Apr 2015 14:48:54 -0700 Subject: [PATCH 079/212] HDFS-8156. Add/implement necessary APIs even we just have the system default schema. Contributed by Kai Zheng. --- .../hadoop/io/erasurecode/ECSchema.java | 173 +++++++++++------- .../hadoop/io/erasurecode/TestECSchema.java | 2 +- .../io/erasurecode/TestSchemaLoader.java | 6 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/protocolPB/PBHelper.java | 2 +- .../hdfs/server/namenode/ECSchemaManager.java | 79 +++++++- .../namenode/ErasureCodingZoneManager.java | 16 +- .../hdfs/server/namenode/FSNamesystem.java | 29 ++- .../org/apache/hadoop/hdfs/TestECSchemas.java | 5 +- .../hadoop/hdfs/TestErasureCodingZones.java | 45 +++-- 10 files changed, 249 insertions(+), 111 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index 32077f6630667..f058ea71888ed 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -18,6 +18,7 @@ package org.apache.hadoop.io.erasurecode; import java.util.Collections; +import java.util.HashMap; import java.util.Map; /** @@ -30,55 +31,80 @@ public final class ECSchema { public static final String CHUNK_SIZE_KEY = "chunkSize"; public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K - private String schemaName; - private String codecName; - private Map options; - private int numDataUnits; - private int numParityUnits; - private int chunkSize; + /** + * A friendly and understandable name that can mean what's it, also serves as + * the identifier that distinguish it from other schemas. + */ + private final String schemaName; + + /** + * The erasure codec name associated. + */ + private final String codecName; + + /** + * Number of source data units coded + */ + private final int numDataUnits; + + /** + * Number of parity units generated in a coding + */ + private final int numParityUnits; + + /** + * Unit data size for each chunk in a coding + */ + private final int chunkSize; + + /* + * An erasure code can have its own specific advanced parameters, subject to + * itself to interpret these key-value settings. + */ + private final Map extraOptions; /** - * Constructor with schema name and provided options. Note the options may + * Constructor with schema name and provided all options. Note the options may * contain additional information for the erasure codec to interpret further. * @param schemaName schema name - * @param options schema options + * @param allOptions all schema options */ - public ECSchema(String schemaName, Map options) { + public ECSchema(String schemaName, Map allOptions) { assert (schemaName != null && ! schemaName.isEmpty()); this.schemaName = schemaName; - if (options == null || options.isEmpty()) { + if (allOptions == null || allOptions.isEmpty()) { throw new IllegalArgumentException("No schema options are provided"); } - String codecName = options.get(CODEC_NAME_KEY); + this.codecName = allOptions.get(CODEC_NAME_KEY); if (codecName == null || codecName.isEmpty()) { throw new IllegalArgumentException("No codec option is provided"); } - int dataUnits = 0, parityUnits = 0; - try { - if (options.containsKey(NUM_DATA_UNITS_KEY)) { - dataUnits = Integer.parseInt(options.get(NUM_DATA_UNITS_KEY)); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Option value " + - options.get(NUM_DATA_UNITS_KEY) + " for " + NUM_DATA_UNITS_KEY + - " is found. It should be an integer"); + int tmpNumDataUnits = extractIntOption(NUM_DATA_UNITS_KEY, allOptions); + int tmpNumParityUnits = extractIntOption(NUM_PARITY_UNITS_KEY, allOptions); + if (tmpNumDataUnits < 0 || tmpNumParityUnits < 0) { + throw new IllegalArgumentException( + "No good option for numDataUnits or numParityUnits found "); } - - try { - if (options.containsKey(NUM_PARITY_UNITS_KEY)) { - parityUnits = Integer.parseInt(options.get(NUM_PARITY_UNITS_KEY)); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Option value " + - options.get(NUM_PARITY_UNITS_KEY) + " for " + NUM_PARITY_UNITS_KEY + - " is found. It should be an integer"); + this.numDataUnits = tmpNumDataUnits; + this.numParityUnits = tmpNumParityUnits; + + int tmpChunkSize = extractIntOption(CHUNK_SIZE_KEY, allOptions); + if (tmpChunkSize > 0) { + this.chunkSize = tmpChunkSize; + } else { + this.chunkSize = DEFAULT_CHUNK_SIZE; } - initWith(codecName, dataUnits, parityUnits, options); + allOptions.remove(CODEC_NAME_KEY); + allOptions.remove(NUM_DATA_UNITS_KEY); + allOptions.remove(NUM_PARITY_UNITS_KEY); + allOptions.remove(CHUNK_SIZE_KEY); + // After some cleanup + this.extraOptions = Collections.unmodifiableMap(allOptions); } /** @@ -94,48 +120,60 @@ public ECSchema(String schemaName, String codecName, } /** - * Constructor with key parameters provided. Note the options may contain + * Constructor with key parameters provided. Note the extraOptions may contain * additional information for the erasure codec to interpret further. * @param schemaName * @param codecName * @param numDataUnits * @param numParityUnits - * @param options + * @param extraOptions */ - public ECSchema(String schemaName, String codecName, - int numDataUnits, int numParityUnits, - Map options) { + public ECSchema(String schemaName, String codecName, int numDataUnits, + int numParityUnits, Map extraOptions) { + assert (schemaName != null && ! schemaName.isEmpty()); assert (codecName != null && ! codecName.isEmpty()); + assert (numDataUnits > 0 && numParityUnits > 0); this.schemaName = schemaName; - initWith(codecName, numDataUnits, numParityUnits, options); - } - - private void initWith(String codecName, int numDataUnits, int numParityUnits, - Map options) { this.codecName = codecName; this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; - this.options = options != null ? Collections.unmodifiableMap(options) : - Collections.EMPTY_MAP; + if (extraOptions == null) { + extraOptions = new HashMap<>(); + } + + int tmpChunkSize = extractIntOption(CHUNK_SIZE_KEY, extraOptions); + if (tmpChunkSize > 0) { + this.chunkSize = tmpChunkSize; + } else { + this.chunkSize = DEFAULT_CHUNK_SIZE; + } + + extraOptions.remove(CHUNK_SIZE_KEY); + // After some cleanup + this.extraOptions = Collections.unmodifiableMap(extraOptions); + } + + private int extractIntOption(String optionKey, Map options) { + int result = -1; - this.chunkSize = DEFAULT_CHUNK_SIZE; try { - if (this.options.containsKey(CHUNK_SIZE_KEY)) { - this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY)); + if (options.containsKey(optionKey)) { + result = Integer.parseInt(options.get(optionKey)); + if (result <= 0) { + throw new IllegalArgumentException("Bad option value " + result + + " found for " + optionKey); + } } } catch (NumberFormatException e) { throw new IllegalArgumentException("Option value " + - this.options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY + + options.get(optionKey) + " for " + optionKey + " is found. It should be an integer"); } - boolean isFine = numDataUnits > 0 && numParityUnits > 0 && chunkSize > 0; - if (! isFine) { - throw new IllegalArgumentException("Bad codec options are found"); - } + return result; } /** @@ -155,11 +193,11 @@ public String getCodecName() { } /** - * Get erasure coding options - * @return encoding options + * Get extra options specific to a erasure code. + * @return extra options */ - public Map getOptions() { - return options; + public Map getExtraOptions() { + return extraOptions; } /** @@ -194,18 +232,17 @@ public int getChunkSize() { public String toString() { StringBuilder sb = new StringBuilder("ECSchema=["); - sb.append("Name=" + schemaName + ","); - sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ","); - sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ","); - sb.append(CHUNK_SIZE_KEY + "=" + chunkSize + ","); - - for (String opt : options.keySet()) { - boolean skip = (opt.equals(NUM_DATA_UNITS_KEY) || - opt.equals(NUM_PARITY_UNITS_KEY) || - opt.equals(CHUNK_SIZE_KEY)); - if (! skip) { - sb.append(opt + "=" + options.get(opt) + ","); - } + sb.append("Name=" + schemaName + ", "); + sb.append("Codec=" + codecName + ", "); + sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ", "); + sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ", "); + sb.append(CHUNK_SIZE_KEY + "=" + chunkSize + + (extraOptions.isEmpty() ? "" : ", ")); + + int i = 0; + for (String opt : extraOptions.keySet()) { + sb.append(opt + "=" + extraOptions.get(opt) + + (++i < extraOptions.size() ? ", " : "")); } sb.append("]"); @@ -239,14 +276,14 @@ public boolean equals(Object o) { if (!codecName.equals(ecSchema.codecName)) { return false; } - return options.equals(ecSchema.options); + return extraOptions.equals(ecSchema.extraOptions); } @Override public int hashCode() { int result = schemaName.hashCode(); result = 31 * result + codecName.hashCode(); - result = 31 * result + options.hashCode(); + result = 31 * result + extraOptions.hashCode(); result = 31 * result + numDataUnits; result = 31 * result + numParityUnits; result = 31 * result + chunkSize; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java index 4285ef0185fa1..15e672fbb9fed 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java @@ -49,6 +49,6 @@ public void testGoodSchema() { assertEquals(numParityUnits, schema.getNumParityUnits()); assertEquals(chunkSize, schema.getChunkSize()); assertEquals(codec, schema.getCodecName()); - assertEquals(extraOptionValue, schema.getOptions().get(extraOption)); + assertEquals(extraOptionValue, schema.getExtraOptions().get(extraOption)); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java index 6caeedb3b479d..939fa9b861304 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java @@ -59,16 +59,16 @@ public void testLoadSchema() throws Exception { ECSchema schema1 = schemas.get(0); assertEquals("RSk6m3", schema1.getSchemaName()); - assertEquals(3, schema1.getOptions().size()); + assertEquals(0, schema1.getExtraOptions().size()); assertEquals(6, schema1.getNumDataUnits()); assertEquals(3, schema1.getNumParityUnits()); assertEquals("RS", schema1.getCodecName()); ECSchema schema2 = schemas.get(1); assertEquals("RSk10m4", schema2.getSchemaName()); - assertEquals(3, schema2.getOptions().size()); + assertEquals(0, schema2.getExtraOptions().size()); assertEquals(10, schema2.getNumDataUnits()); assertEquals(4, schema2.getNumParityUnits()); assertEquals("RS", schema2.getCodecName()); } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 1acde41811726..b2faac048f16d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -116,3 +116,6 @@ HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. (umamahesh) + + HDFS-8156. Add/implement necessary APIs even we just have the system default + schema. (Kai Zheng via Zhe Zhang) 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 68cfe7f57d49b..3cd3e03d1bcad 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 @@ -3143,7 +3143,7 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { .setCodecName(schema.getCodecName()) .setDataUnits(schema.getNumDataUnits()) .setParityUnits(schema.getNumParityUnits()); - Set> entrySet = schema.getOptions().entrySet(); + Set> entrySet = schema.getExtraOptions().entrySet(); for (Entry entry : entrySet) { builder.addOptions(ECSchemaOptionEntryProto.newBuilder() .setKey(entry.getKey()).setValue(entry.getValue()).build()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java index b001c57fd2b27..2d634986a7db1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java @@ -20,22 +20,62 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.io.erasurecode.ECSchema; +import java.util.Map; +import java.util.TreeMap; + /** - * This manages EC schemas predefined and activated in the system. It loads from - * predefined ones in XML and syncs with persisted ones in NameNode image. + * This manages EC schemas predefined and activated in the system. + * It loads customized schemas and syncs with persisted ones in + * NameNode image. * * This class is instantiated by the FSNamesystem. */ @InterfaceAudience.LimitedPrivate({"HDFS"}) public final class ECSchemaManager { + /** + * TODO: HDFS-8095 + */ private static final int DEFAULT_DATA_BLOCKS = 6; private static final int DEFAULT_PARITY_BLOCKS = 3; private static final String DEFAULT_CODEC_NAME = "rs"; - private static final String DEFAULT_SCHEMA_NAME = "SYS-DEFAULT-RS-6-3"; + private static final String DEFAULT_SCHEMA_NAME = "RS-6-3"; + private static final ECSchema SYS_DEFAULT_SCHEMA = + new ECSchema(DEFAULT_SCHEMA_NAME, + DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS); + + //We may add more later. + private static ECSchema[] SYS_SCHEMAS = new ECSchema[] { + SYS_DEFAULT_SCHEMA + }; + + /** + * All active EC activeSchemas maintained in NN memory for fast querying, + * identified and sorted by its name. + */ + private final Map activeSchemas; - private static ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(DEFAULT_SCHEMA_NAME, - DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS); + ECSchemaManager() { + + this.activeSchemas = new TreeMap(); + for (ECSchema schema : SYS_SCHEMAS) { + activeSchemas.put(schema.getSchemaName(), schema); + } + + /** + * TODO: HDFS-7859 persist into NameNode + * load persistent schemas from image and editlog, which is done only once + * during NameNode startup. This can be done here or in a separate method. + */ + } + + /** + * Get system defined schemas. + * @return system schemas + */ + public static ECSchema[] getSystemSchemas() { + return SYS_SCHEMAS; + } /** * Get system-wide default EC schema, which can be used by default when no @@ -56,7 +96,32 @@ public static boolean isSystemDefault(ECSchema schema) { throw new IllegalArgumentException("Invalid schema parameter"); } - // schema name is the identifier, but for safety we check all properties. - return SYS_DEFAULT_SCHEMA.equals(schema); + // schema name is the identifier. + return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName()); + } + + /** + * Get all EC schemas that's available to use. + * @return all EC schemas + */ + public ECSchema[] getSchemas() { + ECSchema[] results = new ECSchema[activeSchemas.size()]; + return activeSchemas.values().toArray(results); + } + + /** + * Get the EC schema specified by the schema name. + * @param schemaName + * @return EC schema specified by the schema name + */ + public ECSchema getSchema(String schemaName) { + return activeSchemas.get(schemaName); + } + + /** + * Clear and clean up + */ + public void clear() { + activeSchemas.clear(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 3f942279024d4..8cda289849d74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -23,8 +23,6 @@ import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.ECZoneInfo; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; -import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.IOException; @@ -80,9 +78,8 @@ ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { : inode.getXAttrFeature().getXAttrs(); for (XAttr xAttr : xAttrs) { if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { - ECSchemaProto ecSchemaProto; - ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue()); - ECSchema schema = PBHelper.convertECSchema(ecSchemaProto); + String schemaName = new String(xAttr.getValue()); + ECSchema schema = dir.getFSNamesystem().getECSchema(schemaName); return new ECZoneInfo(inode.getFullPathName(), schema); } } @@ -109,13 +106,14 @@ XAttr createErasureCodingZone(String src, ECSchema schema) throw new IOException("Directory " + src + " is already in an " + "erasure coding zone."); } - // TODO HDFS-7859 Need to persist the schema in xattr in efficient way - // As of now storing the protobuf format + + // System default schema will be used since no specified. if (schema == null) { schema = ECSchemaManager.getSystemDefaultSchema(); } - ECSchemaProto schemaProto = PBHelper.convertECSchema(schema); - byte[] schemaBytes = schemaProto.toByteArray(); + + // Now persist the schema name in xattr + byte[] schemaBytes = schema.getSchemaName().getBytes(); final XAttr ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE, schemaBytes); final List xattrs = Lists.newArrayListWithCapacity(1); 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 925f960d63fa7..a2091427873cf 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 @@ -428,6 +428,7 @@ private void logAuditEvent(boolean succeeded, private final BlockManager blockManager; private final SnapshotManager snapshotManager; private final CacheManager cacheManager; + private final ECSchemaManager schemaManager; private final DatanodeStatistics datanodeStatistics; private String nameserviceId; @@ -607,6 +608,7 @@ void clear() { leaseManager.removeAllLeases(); snapshotManager.clearSnapshottableDirs(); cacheManager.clear(); + schemaManager.clear(); setImageLoaded(false); blockManager.clear(); } @@ -846,6 +848,7 @@ static FSNamesystem loadFromDisk(Configuration conf) throws IOException { this.dir = new FSDirectory(this, conf); this.snapshotManager = new SnapshotManager(dir); this.cacheManager = new CacheManager(this, conf, blockManager); + this.schemaManager = new ECSchemaManager(); this.safeMode = new SafeModeInfo(conf); this.topConf = new TopConf(conf); this.auditLoggers = initAuditLoggers(conf); @@ -6616,16 +6619,23 @@ public BlockIdManager getBlockIdManager() { public FSDirectory getFSDirectory() { return dir; } + /** Set the FSDirectory. */ @VisibleForTesting public void setFSDirectory(FSDirectory dir) { this.dir = dir; } + /** @return the cache manager. */ public CacheManager getCacheManager() { return cacheManager; } + /** @return the schema manager. */ + public ECSchemaManager getSchemaManager() { + return schemaManager; + } + @Override // NameNodeMXBean public String getCorruptFiles() { List list = new ArrayList(); @@ -7626,9 +7636,22 @@ ECSchema[] getECSchemas() throws IOException { readLock(); try { checkOperation(OperationCategory.READ); - // TODO HDFS-7866 Need to return all schemas maintained by Namenode - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); - return new ECSchema[] { defaultSchema }; + return schemaManager.getSchemas(); + } finally { + readUnlock(); + } + } + + /** + * Get the ECSchema specified by the name + */ + ECSchema getECSchema(String schemaName) throws IOException { + checkOperation(OperationCategory.READ); + waitForLoadingFSImage(); + readLock(); + try { + checkOperation(OperationCategory.READ); + return schemaManager.getSchema(schemaName); } finally { readUnlock(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java index 07e1359c2ddab..83d208a1a0bf3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java @@ -48,10 +48,7 @@ public void after() { @Test public void testGetECSchemas() throws Exception { ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas(); - // TODO update assertion after HDFS-7866 assertNotNull(ecSchemas); - assertEquals("Should have only one ecSchema", 1, ecSchemas.length); - assertEquals("Returned schemas should have only default schema", - ECSchemaManager.getSystemDefaultSchema(), ecSchemas[0]); + assertTrue("Should have at least one schema", ecSchemas.length > 0); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index 699df4e3227e3..ae2bdd85fb03d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -151,35 +152,49 @@ public void testMoveValidity() throws IOException, InterruptedException { } @Test - public void testGetErasureCodingInfo() throws Exception { + public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { String src = "/ec"; final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); // dir ECInfo before creating ec zone assertNull(fs.getClient().getErasureCodingInfo(src)); // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, null); - verifyErasureCodingInfo(src); + fs.getClient().createErasureCodingZone(src, null); //Default one will be used. + ECSchema sysDefaultSchema = ECSchemaManager.getSystemDefaultSchema(); + verifyErasureCodingInfo(src, sysDefaultSchema); + fs.create(new Path(ecDir, "/child1")).close(); + // verify for the files in ec zone + verifyErasureCodingInfo(src + "/child1", sysDefaultSchema); + } + + @Test + public void testGetErasureCodingInfo() throws Exception { + ECSchema[] sysSchemas = ECSchemaManager.getSystemSchemas(); + assertTrue("System schemas should be of only 1 for now", + sysSchemas.length == 1); + + ECSchema usingSchema = sysSchemas[0]; + String src = "/ec2"; + final Path ecDir = new Path(src); + fs.mkdir(ecDir, FsPermission.getDirDefault()); + // dir ECInfo before creating ec zone + assertNull(fs.getClient().getErasureCodingInfo(src)); + // dir ECInfo after creating ec zone + fs.getClient().createErasureCodingZone(src, usingSchema); + verifyErasureCodingInfo(src, usingSchema); fs.create(new Path(ecDir, "/child1")).close(); // verify for the files in ec zone - verifyErasureCodingInfo(src + "/child1"); + verifyErasureCodingInfo(src + "/child1", usingSchema); } - private void verifyErasureCodingInfo(String src) throws IOException { + private void verifyErasureCodingInfo( + String src, ECSchema usingSchema) throws IOException { ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src); assertNotNull("ECInfo should have been non-null", ecInfo); assertEquals(src, ecInfo.getSrc()); ECSchema schema = ecInfo.getSchema(); assertNotNull(schema); - assertEquals("Default schema should be returned", "RS-6-3", - schema.getSchemaName()); - assertEquals("Default codec(rs) should be returned", "rs", - schema.getCodecName()); - assertEquals("Default numDataUnits should be used", 6, - schema.getNumDataUnits()); - assertEquals("Default numParityUnits should be used", 3, - schema.getNumParityUnits()); - assertEquals("Default chunkSize should be used", - ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize()); + assertEquals("Actually used schema should be equal with target schema", + usingSchema, schema); } } From fcd54ecce2760c6e576b326f67c41119adb46274 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 24 Apr 2015 00:19:12 +0800 Subject: [PATCH 080/212] HDFS-8136. Client gets and uses EC schema when reads and writes a stripping file. Contributed by Kai Sasaki --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 17 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 24 ++- .../hdfs/TestDFSStripedInputStream.java | 175 ++++++++++++++++++ .../hdfs/TestDFSStripedOutputStream.java | 4 +- .../hadoop/hdfs/TestReadStripedFile.java | 1 - 6 files changed, 209 insertions(+), 15 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index b2faac048f16d..8977c46989a84 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -119,3 +119,6 @@ HDFS-8156. Add/implement necessary APIs even we just have the system default schema. (Kai Zheng via Zhe Zhang) + + HDFS-8136. Client gets and uses EC schema when reads and writes a stripping + file. (Kai Sasaki via Kai Zheng) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index d597407305c1b..d0e2b6822d03a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -21,9 +21,9 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.net.NetUtils; @@ -125,13 +125,19 @@ static ReadPortion[] planReadPortions(final int dataBlkNum, return results; } - private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - private final short dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS; - private final short parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS; + private final int cellSize; + private final short dataBlkNum; + private final short parityBlkNum; + private final ECInfo ecInfo; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum) throws IOException { super(dfsClient, src, verifyChecksum); + // ECInfo is restored from NN just before reading striped file. + ecInfo = dfsClient.getErasureCodingInfo(src); + cellSize = ecInfo.getSchema().getChunkSize(); + dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits(); + parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits(); DFSClient.LOG.debug("Creating an striped input stream for file " + src); } @@ -279,9 +285,6 @@ private void waitNextCompletion(CompletionService stripedReadsService, throw new InterruptedException("let's retry"); } - public void setCellSize(int cellSize) { - this.cellSize = cellSize; - } /** * This class represents the portion of I/O associated with each block in the diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 7dc00919b834f..eeb9d7ea96b77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -32,8 +32,8 @@ import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.util.StripedBlockUtil; @@ -61,11 +61,13 @@ public class DFSStripedOutputStream extends DFSOutputStream { /** * Size of each striping cell, must be a multiple of bytesPerChecksum */ - private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final ECInfo ecInfo; + private final int cellSize; private ByteBuffer[] cellBuffers; - private final short numAllBlocks = HdfsConstants.NUM_DATA_BLOCKS - + HdfsConstants.NUM_PARITY_BLOCKS; - private final short numDataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + + private final short numAllBlocks; + private final short numDataBlocks; + private int curIdx = 0; /* bytes written in current block group */ //private long currentBlockGroupBytes = 0; @@ -77,6 +79,10 @@ private StripedDataStreamer getLeadingStreamer() { return streamers.get(0); } + private long getBlockGroupSize() { + return blockSize * numDataBlocks; + } + /** Construct a new output stream for creating a file. */ DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, EnumSet flag, Progressable progress, @@ -84,6 +90,14 @@ private StripedDataStreamer getLeadingStreamer() { throws IOException { super(dfsClient, src, stat, flag, progress, checksum, favoredNodes); DFSClient.LOG.info("Creating striped output stream"); + + // ECInfo is restored from NN just before writing striped files. + ecInfo = dfsClient.getErasureCodingInfo(src); + cellSize = ecInfo.getSchema().getChunkSize(); + numAllBlocks = (short)(ecInfo.getSchema().getNumDataUnits() + + ecInfo.getSchema().getNumParityUnits()); + numDataBlocks = (short)ecInfo.getSchema().getNumDataUnits(); + checkConfiguration(); cellBuffers = new ByteBuffer[numAllBlocks]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java new file mode 100644 index 0000000000000..6af4a7f330dd6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -0,0 +1,175 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; + +public class TestDFSStripedInputStream { + private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + + + private static DistributedFileSystem fs; + private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + static int blockSize = cellSize * stripesPerBlock; + private int mod = 29; + static int numDNs = dataBlocks + parityBlocks + 2; + + private static MiniDFSCluster cluster; + private static Configuration conf; + + @BeforeClass + public static void setup() throws IOException { + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster + = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();; + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + fs = cluster.getFileSystem(); + } + + @AfterClass + public static void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testFileEmpty() throws IOException { + testOneFileUsingDFSStripedInputStream("/EmptyFile", 0); + } + + @Test + public void testFileSmallerThanOneCell1() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1); + } + + @Test + public void testFileSmallerThanOneCell2() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1); + } + + @Test + public void testFileEqualsWithOneCell() throws IOException { + testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize); + } + + @Test + public void testFileSmallerThanOneStripe1() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize * dataBlocks - 1); + } + + @Test + public void testFileSmallerThanOneStripe2() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize + 123); + } + + @Test + public void testFileEqualsWithOneStripe() throws IOException { + testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", cellSize * dataBlocks); + } + + @Test + public void testFileMoreThanOneStripe1() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", cellSize * dataBlocks + 123); + } + + @Test + public void testFileMoreThanOneStripe2() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", cellSize * dataBlocks + + cellSize * dataBlocks + 123); + } + + @Test + public void testFileFullBlockGroup() throws IOException { + testOneFileUsingDFSStripedInputStream("/FullBlockGroup", blockSize * dataBlocks); + } + + @Test + public void testFileMoreThanABlockGroup1() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); + } + + @Test + public void testFileMoreThanABlockGroup2() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123); + } + + + @Test + public void testFileMoreThanABlockGroup3() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123); + } + + private byte[] generateBytes(int cnt) { + byte[] bytes = new byte[cnt]; + for (int i = 0; i < cnt; i++) { + bytes[i] = getByte(i); + } + return bytes; + } + + private byte getByte(long pos) { + return (byte) (pos % mod + 1); + } + + private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) + throws IOException { + Path TestPath = new Path(src); + byte[] bytes = generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); + + //check file length + FileStatus status = fs.getFileStatus(TestPath); + long fileLength = status.getLen(); + Assert.assertEquals("File length should be the same", + writeBytes, fileLength); + + DFSStripedInputStream dis = new DFSStripedInputStream( + fs.getClient(), src, true); + try { + byte[] buf = new byte[writeBytes + 100]; + int readLen = dis.read(0, buf, 0, buf.length); + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + for (int i = 0; i < writeBytes; i++) { + Assert.assertEquals("Byte at i should be the same", + getByte(i), buf[i]); + } + } finally { + dis.close(); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index c2131834b8122..26f6d2c22dc28 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -159,7 +159,7 @@ private void testOneFile(String src, int writeBytes) throws IOException { // check file length FileStatus status = fs.getFileStatus(testPath); Assert.assertEquals(writeBytes, status.getLen()); - + checkData(src, writeBytes); } @@ -236,7 +236,7 @@ void checkData(String src, int writeBytes) throws IOException { cellSize, dataBlockBytes, parityBlockBytes); } } - + static void verifyParity(final long size, final int cellSize, byte[][] dataBytes, byte[][] parityBytes) { // verify the parity blocks diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java index 90488c19eac66..b0631cedb6d9b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java @@ -121,7 +121,6 @@ public void testPread() throws Exception { } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); - in.setCellSize(CELLSIZE); int readSize = BLOCKSIZE; byte[] readBuffer = new byte[readSize]; int ret = in.read(0, readBuffer, 0, readSize); From eb612b0b70f1439194d3c1cdf151f3301e88522b Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 23 Apr 2015 15:43:04 -0700 Subject: [PATCH 081/212] HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last stripe is at the block group boundary. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 51 +++++++++---------- .../hdfs/TestDFSStripedOutputStream.java | 6 +++ 3 files changed, 34 insertions(+), 28 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8977c46989a84..48791b134a69f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -121,4 +121,7 @@ schema. (Kai Zheng via Zhe Zhang) HDFS-8136. Client gets and uses EC schema when reads and writes a stripping - file. (Kai Sasaki via Kai Zheng) \ No newline at end of file + file. (Kai Sasaki via Kai Zheng) + + HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last + stripe is at the block group boundary. (jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index eeb9d7ea96b77..245dfc10b6e9a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -36,7 +36,6 @@ 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.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.util.DataChecksum; @@ -278,14 +277,6 @@ private int stripeDataSize() { return numDataBlocks * cellSize; } - private long getCurrentBlockGroupBytes() { - long sum = 0; - for (int i = 0; i < numDataBlocks; i++) { - sum += streamers.get(i).getBytesCurBlock(); - } - return sum; - } - private void notSupported(String headMsg) throws IOException{ throw new IOException( @@ -347,37 +338,43 @@ protected void closeThreads(boolean force) throws IOException { } } + /** + * Simply add bytesCurBlock together. Note that this result is not accurately + * the size of the block group. + */ + private long getCurrentSumBytes() { + long sum = 0; + for (int i = 0; i < numDataBlocks; i++) { + sum += streamers.get(i).getBytesCurBlock(); + } + return sum; + } + private void writeParityCellsForLastStripe() throws IOException { - final long currentBlockGroupBytes = getCurrentBlockGroupBytes(); - long parityBlkSize = StripedBlockUtil.getInternalBlockLength( - currentBlockGroupBytes, cellSize, numDataBlocks, - numDataBlocks + 1); - if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) { + final long currentBlockGroupBytes = getCurrentSumBytes(); + if (currentBlockGroupBytes % stripeDataSize() == 0) { return; } - int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize : - (int) (parityBlkSize % cellSize); + long firstCellSize = getLeadingStreamer().getBytesCurBlock() % cellSize; + long parityCellSize = firstCellSize > 0 && firstCellSize < cellSize ? + firstCellSize : cellSize; for (int i = 0; i < numAllBlocks; i++) { - long internalBlkLen = StripedBlockUtil.getInternalBlockLength( - currentBlockGroupBytes, cellSize, numDataBlocks, i); // Pad zero bytes to make all cells exactly the size of parityCellSize // If internal block is smaller than parity block, pad zero bytes. // Also pad zero bytes to all parity cells - if (internalBlkLen < parityBlkSize || i >= numDataBlocks) { - int position = cellBuffers[i].position(); - assert position <= parityCellSize : "If an internal block is smaller" + - " than parity block, then its last cell should be small than last" + - " parity cell"; - for (int j = 0; j < parityCellSize - position; j++) { - cellBuffers[i].put((byte) 0); - } + int position = cellBuffers[i].position(); + assert position <= parityCellSize : "If an internal block is smaller" + + " than parity block, then its last cell should be small than last" + + " parity cell"; + for (int j = 0; j < parityCellSize - position; j++) { + cellBuffers[i].put((byte) 0); } cellBuffers[i].flip(); } encode(cellBuffers); - //write parity cells + // write parity cells curIdx = numDataBlocks; refreshStreamer(); for (int i = numDataBlocks; i < numAllBlocks; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 26f6d2c22dc28..5ce94ee1a41cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -114,6 +114,12 @@ public void testFileMoreThanOneStripe2() throws IOException { + cellSize * dataBlocks + 123); } + @Test + public void testFileLessThanFullBlockGroup() throws IOException { + testOneFile("/LessThanFullBlockGroup", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); + } + @Test public void testFileFullBlockGroup() throws IOException { testOneFile("/FullBlockGroup", blockSize * dataBlocks); From 3f2c6938f1d157ff587826bd91b09906e60b53f4 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 23 Apr 2015 15:48:21 -0700 Subject: [PATCH 082/212] HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream. Contributed by Yi Liu. --- .../main/java/org/apache/hadoop/fs/FSOutputSummer.java | 4 ++++ .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../org/apache/hadoop/hdfs/DFSStripedOutputStream.java | 10 ++++++++++ 3 files changed, 17 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java index bdc55853d8beb..a8a74945e5bf5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java @@ -196,6 +196,10 @@ protected int getChecksumSize() { return sum.getChecksumSize(); } + protected DataChecksum getDataChecksum() { + return sum; + } + protected TraceScope createWriteTraceScope() { return NullScope.INSTANCE; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 48791b134a69f..9357e23fddd19 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -125,3 +125,6 @@ HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last stripe is at the block group boundary. (jing9) + + HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream. + (Yi Liu via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 245dfc10b6e9a..68422675a1ec6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -62,6 +62,8 @@ public class DFSStripedOutputStream extends DFSOutputStream { */ private final ECInfo ecInfo; private final int cellSize; + // checksum buffer, we only need to calculate checksum for parity blocks + private byte[] checksumBuf; private ByteBuffer[] cellBuffers; private final short numAllBlocks; @@ -99,6 +101,7 @@ private long getBlockGroupSize() { checkConfiguration(); + checksumBuf = new byte[getChecksumSize() * (cellSize / bytesPerChecksum)]; cellBuffers = new ByteBuffer[numAllBlocks]; List> stripeBlocks = new ArrayList<>(); @@ -179,6 +182,10 @@ private void encode(ByteBuffer[] buffers) { private List generatePackets(ByteBuffer byteBuffer) throws IOException{ List packets = new ArrayList<>(); + assert byteBuffer.hasArray(); + getDataChecksum().calculateChunkedSums(byteBuffer.array(), 0, + byteBuffer.remaining(), checksumBuf, 0); + int ckOff = 0; while (byteBuffer.remaining() > 0) { DFSPacket p = createPacket(packetSize, chunksPerPacket, streamer.getBytesCurBlock(), @@ -186,6 +193,9 @@ private List generatePackets(ByteBuffer byteBuffer) int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum; int toWrite = byteBuffer.remaining() > maxBytesToPacket ? maxBytesToPacket: byteBuffer.remaining(); + int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * getChecksumSize(); + p.writeChecksum(checksumBuf, ckOff, ckLen); + ckOff += ckLen; p.writeData(byteBuffer, toWrite); streamer.incBytesCurBlock(toWrite); packets.add(p); From f5d4a95ef5fc01eb2b93fb79798ad2357cfc2ba0 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 24 Apr 2015 09:30:38 -0700 Subject: [PATCH 083/212] HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause block id conflicts. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../SequentialBlockGroupIdGenerator.java | 39 +++++++------ .../SequentialBlockIdGenerator.java | 2 +- .../hdfs/TestDFSStripedInputStream.java | 57 ++++++++++--------- .../server/namenode/TestAddStripedBlocks.java | 21 +++++++ 5 files changed, 77 insertions(+), 45 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9357e23fddd19..cf41a9b2ae1c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -128,3 +128,6 @@ HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream. (Yi Liu via jing9) + + HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause + block id conflicts (Jing Zhao via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java index e9e22ee46d539..de8e379389be5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java @@ -19,9 +19,11 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.util.SequentialNumber; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_GROUP_INDEX_MASK; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_BLOCKS_IN_GROUP; + /** * Generate the next valid block group ID by incrementing the maximum block * group ID allocated so far, with the first 2^10 block group IDs reserved. @@ -34,6 +36,9 @@ * bits (n+2) to (64-m) represent the ID of its block group, while the last m * bits represent its index of the group. The value m is determined by the * maximum number of blocks in a group (MAX_BLOCKS_IN_GROUP). + * + * Note that the {@link #nextValue()} methods requires external lock to + * guarantee IDs have no conflicts. */ @InterfaceAudience.Private public class SequentialBlockGroupIdGenerator extends SequentialNumber { @@ -47,32 +52,30 @@ public class SequentialBlockGroupIdGenerator extends SequentialNumber { @Override // NumberGenerator public long nextValue() { - // Skip to next legitimate block group ID based on the naming protocol - while (super.getCurrentValue() % HdfsConstants.MAX_BLOCKS_IN_GROUP > 0) { - super.nextValue(); - } + skipTo((getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + MAX_BLOCKS_IN_GROUP); // Make sure there's no conflict with existing random block IDs - while (hasValidBlockInRange(super.getCurrentValue())) { - super.skipTo(super.getCurrentValue() + - HdfsConstants.MAX_BLOCKS_IN_GROUP); + final Block b = new Block(getCurrentValue()); + while (hasValidBlockInRange(b)) { + skipTo(getCurrentValue() + MAX_BLOCKS_IN_GROUP); + b.setBlockId(getCurrentValue()); } - if (super.getCurrentValue() >= 0) { - BlockManager.LOG.warn("All negative block group IDs are used, " + - "growing into positive IDs, " + - "which might conflict with non-erasure coded blocks."); + if (b.getBlockId() >= 0) { + throw new IllegalStateException("All negative block group IDs are used, " + + "growing into positive IDs, " + + "which might conflict with non-erasure coded blocks."); } - return super.getCurrentValue(); + return getCurrentValue(); } /** - * - * @param id The starting ID of the range + * @param b A block object whose id is set to the starting point for check * @return true if any ID in the range * {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a file */ - private boolean hasValidBlockInRange(long id) { - for (int i = 0; i < HdfsConstants.MAX_BLOCKS_IN_GROUP; i++) { - Block b = new Block(id + i); + private boolean hasValidBlockInRange(Block b) { + final long id = b.getBlockId(); + for (int i = 0; i < MAX_BLOCKS_IN_GROUP; i++) { + b.setBlockId(id + i); if (blockManager.getBlockCollection(b) != null) { return true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java index c97de4b806aaf..607478431a051 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java @@ -54,7 +54,7 @@ public long nextValue() { b.setBlockId(super.nextValue()); } if (b.getBlockId() < 0) { - BlockManager.LOG.warn("All positive block IDs are used, " + + throw new IllegalStateException("All positive block IDs are used, " + "wrapping to negative IDs, " + "which might conflict with erasure coded block groups."); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 6af4a7f330dd6..73c735093f9a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -22,10 +22,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -40,18 +38,15 @@ public class TestDFSStripedInputStream { private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final static int stripesPerBlock = 4; static int blockSize = cellSize * stripesPerBlock; - private int mod = 29; static int numDNs = dataBlocks + parityBlocks + 2; private static MiniDFSCluster cluster; - private static Configuration conf; @BeforeClass public static void setup() throws IOException { - conf = new Configuration(); + Configuration conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); - cluster - = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();; + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } @@ -85,43 +80,56 @@ public void testFileEqualsWithOneCell() throws IOException { @Test public void testFileSmallerThanOneStripe1() throws IOException { - testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize * dataBlocks - 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", + cellSize * dataBlocks - 1); } @Test public void testFileSmallerThanOneStripe2() throws IOException { - testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize + 123); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", + cellSize + 123); } @Test public void testFileEqualsWithOneStripe() throws IOException { - testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", cellSize * dataBlocks); + testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", + cellSize * dataBlocks); } @Test public void testFileMoreThanOneStripe1() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", cellSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", + cellSize * dataBlocks + 123); } @Test public void testFileMoreThanOneStripe2() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", cellSize * dataBlocks - + cellSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", + cellSize * dataBlocks + cellSize * dataBlocks + 123); + } + + @Test + public void testLessThanFullBlockGroup() throws IOException { + testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); } @Test public void testFileFullBlockGroup() throws IOException { - testOneFileUsingDFSStripedInputStream("/FullBlockGroup", blockSize * dataBlocks); + testOneFileUsingDFSStripedInputStream("/FullBlockGroup", + blockSize * dataBlocks); } @Test public void testFileMoreThanABlockGroup1() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", + blockSize * dataBlocks + 123); } @Test public void testFileMoreThanABlockGroup2() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", + blockSize * dataBlocks + cellSize+ 123); } @@ -141,35 +149,32 @@ private byte[] generateBytes(int cnt) { } private byte getByte(long pos) { + final int mod = 29; return (byte) (pos % mod + 1); } private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) throws IOException { - Path TestPath = new Path(src); + Path testPath = new Path(src); byte[] bytes = generateBytes(writeBytes); - DFSTestUtil.writeFile(fs, TestPath, new String(bytes)); + DFSTestUtil.writeFile(fs, testPath, new String(bytes)); //check file length - FileStatus status = fs.getFileStatus(TestPath); + FileStatus status = fs.getFileStatus(testPath); long fileLength = status.getLen(); Assert.assertEquals("File length should be the same", writeBytes, fileLength); - DFSStripedInputStream dis = new DFSStripedInputStream( - fs.getClient(), src, true); - try { + try (DFSStripedInputStream dis = + new DFSStripedInputStream(fs.getClient(), src, true)) { byte[] buf = new byte[writeBytes + 100]; int readLen = dis.read(0, buf, 0, buf.length); readLen = readLen >= 0 ? readLen : 0; Assert.assertEquals("The length of file should be the same to write size", writeBytes, readLen); for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at i should be the same", - getByte(i), buf[i]); + Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]); } - } finally { - dis.close(); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 6bb1162adedeb..d03e938b5f1c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -82,6 +82,27 @@ public void tearDown() { } } + /** + * Make sure the IDs of striped blocks do not conflict + */ + @Test + public void testAllocateBlockId() throws Exception { + Path testPath = new Path("/testfile"); + // create a file while allocates a new block + DFSTestUtil.writeFile(dfs, testPath, "hello, world!"); + LocatedBlocks lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0); + final long firstId = lb.get(0).getBlock().getBlockId(); + // delete the file + dfs.delete(testPath, true); + + // allocate a new block, and make sure the new block's id does not conflict + // with the previous one + DFSTestUtil.writeFile(dfs, testPath, "hello again"); + lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0); + final long secondId = lb.get(0).getBlock().getBlockId(); + Assert.assertEquals(firstId + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId); + } + @Test public void testAddStripedBlock() throws Exception { final Path file = new Path("/file1"); From 89d33785780f98a58e1e81eca2c27165840475df Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 24 Apr 2015 22:36:15 -0700 Subject: [PATCH 084/212] HDFS-8033. Erasure coding: stateful (non-positional) read from files in striped layout. Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSInputStream.java | 55 ++-- .../hadoop/hdfs/DFSStripedInputStream.java | 311 +++++++++++++++++- .../hdfs/TestDFSStripedInputStream.java | 43 +++ .../hadoop/hdfs/TestReadStripedFile.java | 110 ++++++- 5 files changed, 465 insertions(+), 57 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index cf41a9b2ae1c6..e8db485ef8e69 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -131,3 +131,6 @@ HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause block id conflicts (Jing Zhao via Zhe Zhang) + + HDFS-8033. Erasure coding: stateful (non-positional) read from files in + striped layout (Zhe Zhang) 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 ecf74f7abbe2e..6649f4c87ca1d 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 @@ -96,34 +96,34 @@ public class DFSInputStream extends FSInputStream public static boolean tcpReadsDisabledForTesting = false; private long hedgedReadOpsLoopNumForTesting = 0; protected final DFSClient dfsClient; - private AtomicBoolean closed = new AtomicBoolean(false); - private final String src; - private final boolean verifyChecksum; + protected AtomicBoolean closed = new AtomicBoolean(false); + protected final String src; + protected final boolean verifyChecksum; // state by stateful read only: // (protected by lock on this) ///// private DatanodeInfo currentNode = null; - private LocatedBlock currentLocatedBlock = null; - private long pos = 0; - private long blockEnd = -1; + protected LocatedBlock currentLocatedBlock = null; + protected long pos = 0; + protected long blockEnd = -1; private BlockReader blockReader = null; //// // state shared by stateful and positional read: // (protected by lock on infoLock) //// - private LocatedBlocks locatedBlocks = null; + protected LocatedBlocks locatedBlocks = null; private long lastBlockBeingWrittenLength = 0; private FileEncryptionInfo fileEncryptionInfo = null; - private CachingStrategy cachingStrategy; + protected CachingStrategy cachingStrategy; //// - private final ReadStatistics readStatistics = new ReadStatistics(); + protected final ReadStatistics readStatistics = new ReadStatistics(); // lock for state shared between read and pread // Note: Never acquire a lock on with this lock held to avoid deadlocks // (it's OK to acquire this lock when the lock on is held) - private final Object infoLock = new Object(); + protected final Object infoLock = new Object(); /** * Track the ByteBuffers that we have handed out to readers. @@ -240,7 +240,7 @@ void clear() { * back to the namenode to get a new list of block locations, and is * capped at maxBlockAcquireFailures */ - private int failures = 0; + protected int failures = 0; /* XXX Use of CocurrentHashMap is temp fix. Need to fix * parallel accesses to DFSInputStream (through ptreads) properly */ @@ -477,7 +477,7 @@ else if (offset >= locatedBlocks.getFileLength()) { } /** Fetch a block from namenode and cache it */ - private void fetchBlockAt(long offset) throws IOException { + protected void fetchBlockAt(long offset) throws IOException { synchronized(infoLock) { int targetBlockIdx = locatedBlocks.findBlock(offset); if (targetBlockIdx < 0) { // block is not cached @@ -580,7 +580,7 @@ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException { } // Will be getting a new BlockReader. - closeCurrentBlockReader(); + closeCurrentBlockReaders(); // // Connect to best DataNode for desired Block, with potential offset @@ -621,7 +621,7 @@ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException { return chosenNode; } catch (IOException ex) { if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { - DFSClient.LOG.info("Will fetch a new encryption key and retry, " + DFSClient.LOG.info("Will fetch a new encryption key and retry, " + "encryption key was invalid when connecting to " + targetAddr + " : " + ex); // The encryption key used is invalid. @@ -697,7 +697,7 @@ public void accept(ByteBuffer k, Object v) { "unreleased ByteBuffers allocated by read(). " + "Please release " + builder.toString() + "."); } - closeCurrentBlockReader(); + closeCurrentBlockReaders(); super.close(); } @@ -719,7 +719,7 @@ public int doRead(BlockReader blockReader, int off, int len) throws ChecksumException, IOException; } - private void updateReadStatistics(ReadStatistics readStatistics, + protected void updateReadStatistics(ReadStatistics readStatistics, int nRead, BlockReader blockReader) { if (nRead <= 0) return; synchronized(infoLock) { @@ -755,7 +755,7 @@ public int doRead(BlockReader blockReader, int off, int len) /** * Used to read bytes into a user-supplied ByteBuffer */ - private class ByteBufferStrategy implements ReaderStrategy { + protected class ByteBufferStrategy implements ReaderStrategy { final ByteBuffer buf; ByteBufferStrategy(ByteBuffer buf) { this.buf = buf; @@ -771,6 +771,9 @@ public int doRead(BlockReader blockReader, int off, int len) int ret = blockReader.read(buf); success = true; updateReadStatistics(readStatistics, ret, blockReader); + if (ret == 0) { + DFSClient.LOG.warn("zero"); + } return ret; } finally { if (!success) { @@ -838,7 +841,7 @@ private synchronized int readBuffer(ReaderStrategy reader, int off, int len, } } - private synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException { + protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException { dfsClient.checkOpen(); if (closed.get()) { throw new IOException("Stream closed"); @@ -927,7 +930,7 @@ public synchronized int read(final ByteBuffer buf) throws IOException { /** * Add corrupted block replica into map. */ - private void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node, + protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node, Map> corruptedBlockMap) { Set dnSet = null; if((corruptedBlockMap.containsKey(blk))) { @@ -999,7 +1002,7 @@ private DNAddrPair chooseDataNode(LocatedBlock block, * @param ignoredNodes Do not choose nodes in this array (may be null) * @return The DNAddrPair of the best node. Null if no node can be chosen. */ - private DNAddrPair getBestNodeDNAddrPair(LocatedBlock block, + protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block, Collection ignoredNodes) { DatanodeInfo[] nodes = block.getLocations(); StorageType[] storageTypes = block.getStorageTypes(); @@ -1368,7 +1371,7 @@ private void cancelAll(List> futures) { * @return true if block access token has expired or invalid and it should be * refetched */ - private static boolean tokenRefetchNeeded(IOException ex, + protected static boolean tokenRefetchNeeded(IOException ex, InetSocketAddress targetAddr) { /* * Get a new access token and retry. Retry is needed in 2 cases. 1) @@ -1475,7 +1478,7 @@ private int pread(long position, byte[] buffer, int offset, int length) * @param corruptedBlockMap map of corrupted blocks * @param dataNodeCount number of data nodes who contains the block replicas */ - private void reportCheckSumFailure( + protected void reportCheckSumFailure( Map> corruptedBlockMap, int dataNodeCount) { if (corruptedBlockMap.isEmpty()) { @@ -1672,7 +1675,7 @@ public FileEncryptionInfo getFileEncryptionInfo() { } } - private void closeCurrentBlockReader() { + protected void closeCurrentBlockReaders() { if (blockReader == null) return; // Close the current block reader so that the new caching settings can // take effect immediately. @@ -1692,7 +1695,7 @@ public synchronized void setReadahead(Long readahead) this.cachingStrategy = new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build(); } - closeCurrentBlockReader(); + closeCurrentBlockReaders(); } @Override @@ -1702,7 +1705,7 @@ public synchronized void setDropBehind(Boolean dropBehind) this.cachingStrategy = new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build(); } - closeCurrentBlockReader(); + closeCurrentBlockReaders(); } /** @@ -1860,6 +1863,6 @@ public synchronized void releaseBuffer(ByteBuffer buffer) { @Override public synchronized void unbuffer() { - closeCurrentBlockReader(); + closeCurrentBlockReaders(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index d0e2b6822d03a..fe9e101c26f2e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -18,20 +18,21 @@ package org.apache.hadoop.hdfs; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.StorageType; -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.protocol.LocatedStripedBlock; -import org.apache.hadoop.hdfs.protocol.ECInfo; -import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; +import org.apache.hadoop.hdfs.protocol.*; +import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.token.Token; import org.apache.htrace.Span; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; import java.io.IOException; +import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -125,6 +126,9 @@ static ReadPortion[] planReadPortions(final int dataBlkNum, return results; } + private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS; + private BlockReader[] blockReaders = null; + private DatanodeInfo[] currentNodes = null; private final int cellSize; private final short dataBlkNum; private final short parityBlkNum; @@ -143,13 +147,285 @@ static ReadPortion[] planReadPortions(final int dataBlkNum, @Override public synchronized int read(final ByteBuffer buf) throws IOException { - throw new UnsupportedActionException("Stateful read is not supported"); + ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf); + TraceScope scope = + dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src); + try { + return readWithStrategy(byteBufferReader, 0, buf.remaining()); + } finally { + scope.close(); + } + } + + /** + * When seeking into a new block group, create blockReader for each internal + * block in the group. + */ + @VisibleForTesting + private synchronized DatanodeInfo[] blockSeekTo(long target) + throws IOException { + if (target >= getFileLength()) { + throw new IOException("Attempted to read past end of file"); + } + + // Will be getting a new BlockReader. + closeCurrentBlockReaders(); + + // Connect to best DataNode for desired Block, with potential offset + DatanodeInfo[] chosenNodes = new DatanodeInfo[groupSize]; + int refetchToken = 1; // only need to get a new access token once + int refetchEncryptionKey = 1; // only need to get a new encryption key once + + // Compute desired striped block group + LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target); + + // Update current position + this.pos = target; + this.blockEnd = targetBlockGroup.getStartOffset() + + targetBlockGroup.getBlockSize() - 1; + + long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset(); + LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup( + targetBlockGroup, cellSize, dataBlkNum, parityBlkNum); + // The purpose is to get start offset into each block + ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, + offsetIntoBlockGroup, 0, 0); + while (true) { + int i = 0; + InetSocketAddress targetAddr = null; + try { + blockReaders = new BlockReader[groupSize]; + for (i = 0; i < groupSize; i++) { + LocatedBlock targetBlock = targetBlocks[i]; + if (targetBlock == null) { + continue; + } + long offsetIntoBlock = readPortions[i].startOffsetInBlock; + DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); + chosenNodes[i] = retval.info; + targetAddr = retval.addr; + StorageType storageType = retval.storageType; + + ExtendedBlock blk = targetBlock.getBlock(); + Token accessToken = targetBlock.getBlockToken(); + CachingStrategy curCachingStrategy; + boolean shortCircuitForbidden; + synchronized(infoLock) { + curCachingStrategy = cachingStrategy; + shortCircuitForbidden = shortCircuitForbidden(); + } + blockReaders[i] = new BlockReaderFactory(dfsClient.getConf()). + setInetSocketAddress(targetAddr). + setRemotePeerFactory(dfsClient). + setDatanodeInfo(chosenNodes[i]). + setStorageType(storageType). + setFileName(src). + setBlock(blk). + setBlockToken(accessToken). + setStartOffset(offsetIntoBlock). + setVerifyChecksum(verifyChecksum). + setClientName(dfsClient.clientName). + setLength(blk.getNumBytes() - offsetIntoBlock). + setCachingStrategy(curCachingStrategy). + setAllowShortCircuitLocalReads(!shortCircuitForbidden). + setClientCacheContext(dfsClient.getClientContext()). + setUserGroupInformation(dfsClient.ugi). + setConfiguration(dfsClient.getConfiguration()). + build(); + } + currentLocatedBlock = targetBlockGroup; + return chosenNodes; + } catch (IOException ex) { + // Retry in case of encryption key or token exceptions. Otherwise throw + // IOException: since each internal block is singly replicated, it's + // not meaningful trying to locate another replica. + if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { + DFSClient.LOG.info("Will fetch a new encryption key and retry, " + + "encryption key was invalid when connecting to " + targetAddr + + " : " + ex); + // The encryption key used is invalid. + refetchEncryptionKey--; + dfsClient.clearDataEncryptionKey(); + } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) { + refetchToken--; + fetchBlockAt(target); + } else { + DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block" + + ", add to deadNodes and continue. " + ex, ex); + // Put chosen node into dead list and throw exception + addToDeadNodes(chosenNodes[i]); + throw ex; + } + } + } + } + + /** + * Extend the super method with the logic of switching between cells. + * When reaching the end of a cell, proceed to the next cell and read it + * with the next blockReader. + */ + @Override + protected void closeCurrentBlockReaders() { + if (blockReaders == null || blockReaders.length == 0) { + return; + } + for (int i = 0; i < groupSize; i++) { + if (blockReaders[i] == null) { + continue; + } + try { + blockReaders[i].close(); + } catch (IOException e) { + DFSClient.LOG.error("error closing blockReader", e); + } + blockReaders[i] = null; + } + blockEnd = -1; } @Override - public synchronized int read(final byte buf[], int off, int len) + protected synchronized int readWithStrategy(ReaderStrategy strategy, + int off, int len) throws IOException { + dfsClient.checkOpen(); + if (closed.get()) { + throw new IOException("Stream closed"); + } + Map> corruptedBlockMap + = new HashMap<>(); + failures = 0; + if (pos < getFileLength()) { + int retries = 2; + /** Index of the target block in a stripe to read from */ + int idxInGroup = (int) ((pos / cellSize) % dataBlkNum); + while (retries > 0) { + try { + // currentNode can be left as null if previous read had a checksum + // error on the same block. See HDFS-3067 + if (pos > blockEnd || currentNodes == null) { + currentNodes = blockSeekTo(pos); + } + int realLen = (int) Math.min(len, (blockEnd - pos + 1L)); + synchronized(infoLock) { + if (locatedBlocks.isLastBlockComplete()) { + realLen = (int) Math.min(realLen, + locatedBlocks.getFileLength() - pos); + } + } + + /** Number of bytes already read into buffer */ + int result = 0; + while (result < realLen) { + /** + * Temporary position into the file; {@link pos} might not proceed + * to this temporary position in case of exceptions. + */ + long tmpPos = pos + result; + /** Start and end offsets of a cell in the file */ + long cellStart = (tmpPos / cellSize) * cellSize; + long cellEnd = cellStart + cellSize - 1; + + /** Number of bytes to read from the current cell */ + int realLenInCell = (int) Math.min(realLen - result, + cellEnd - tmpPos + 1L); + assert realLenInCell > 0 : "Temporary position shouldn't be " + + "after cellEnd"; + // Read from one blockReader up to cell boundary + int cellRet = readBuffer(blockReaders[idxInGroup], + currentNodes[idxInGroup], strategy, off + result, + realLenInCell); + if (cellRet >= 0) { + result += cellRet; + if (cellRet < realLenInCell) { + // A short read indicates the current blockReader buffer is + // already drained. Should return the read call. Otherwise + // should proceed to the next cell. + break; + } + } else { + // got a EOS from reader though we expect more data on it. + throw new IOException("Unexpected EOS from the reader"); + } + idxInGroup = (idxInGroup + 1) % dataBlkNum; + } + + pos += result; + + if (dfsClient.stats != null) { + dfsClient.stats.incrementBytesRead(result); + } + return result; + } catch (ChecksumException ce) { + throw ce; + } catch (IOException e) { + if (retries == 1) { + DFSClient.LOG.warn("DFS Read", e); + } + blockEnd = -1; + if (currentNodes[idxInGroup] != null) { + addToDeadNodes(currentNodes[idxInGroup]); + } + if (--retries == 0) { + throw e; + } + } finally { + // Check if need to report block replicas corruption either read + // was successful or ChecksumException occured. + reportCheckSumFailure(corruptedBlockMap, + currentLocatedBlock.getLocations().length); + } + } + } + return -1; + } + + private synchronized int readBuffer(BlockReader blockReader, + DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len) + throws IOException { + IOException ioe; + while (true) { + try { + return readerStrategy.doRead(blockReader, off, len); + } catch ( ChecksumException ce ) { + DFSClient.LOG.warn("Found Checksum error for " + + getCurrentBlock() + " from " + currentNode + + " at " + ce.getPos()); + // If current block group is corrupt, it's meaningless to retry. + // TODO: this should trigger decoding logic (HDFS-7678) + throw ce; + } catch ( IOException e ) { + ioe = e; + } + + boolean sourceFound = seekToBlockSource(pos); + if (!sourceFound) { + throw ioe; + } + } + } + + private boolean seekToBlockSource(long targetPos) throws IOException { - throw new UnsupportedActionException("Stateful read is not supported"); + currentNodes = blockSeekTo(targetPos); + return true; + } + + protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy { + ByteBufferStrategy(ByteBuffer buf) { + super(buf); + } + + @Override + public int doRead(BlockReader blockReader, int off, int len) + throws ChecksumException, IOException { + int oldlimit = buf.limit(); + if (buf.remaining() > len) { + buf.limit(buf.position() + len); + } + int ret = super.doRead(blockReader, off, len); + buf.limit(oldlimit); + return ret; + } } /** @@ -188,8 +464,11 @@ protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { dataBlkNum, idx); } - private LocatedBlock getBlockGroupAt(long offset) throws IOException { - return super.getBlockAt(offset); + private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException { + LocatedBlock lb = super.getBlockAt(offset); + assert lb instanceof LocatedStripedBlock : "NameNode" + + " should return a LocatedStripedBlock for a striped file"; + return (LocatedStripedBlock)lb; } /** @@ -206,10 +485,8 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, int len = (int) (end - start + 1); // Refresh the striped block group - LocatedBlock block = getBlockGroupAt(blockStartOffset); - assert block instanceof LocatedStripedBlock : "NameNode" + - " should return a LocatedStripedBlock for a striped file"; - LocatedStripedBlock blockGroup = (LocatedStripedBlock) block; + LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset); + // Planning the portion of I/O for each shard ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start, @@ -308,7 +585,7 @@ static class ReadPortion { * +------------------------------------------------------+ */ private long startOffsetInBlock = 0; - private long readLength = 0; + private int readLength = 0; private final List offsetsInBuf = new ArrayList<>(); private final List lengths = new ArrayList<>(); @@ -328,7 +605,7 @@ int[] getLengths() { return lens; } - long getReadLength() { + int getReadLength() { return readLength; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 73c735093f9a0..cf109818c3369 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -28,6 +28,7 @@ import org.junit.Test; import java.io.IOException; +import java.nio.ByteBuffer; public class TestDFSStripedInputStream { private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; @@ -165,6 +166,7 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) Assert.assertEquals("File length should be the same", writeBytes, fileLength); + // pread try (DFSStripedInputStream dis = new DFSStripedInputStream(fs.getClient(), src, true)) { byte[] buf = new byte[writeBytes + 100]; @@ -176,5 +178,46 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]); } } + + // stateful read with byte array + try (DFSStripedInputStream dis = + new DFSStripedInputStream(fs.getClient(), src, true)) { + byte[] buf = new byte[writeBytes + 100]; + int readLen = 0; + int ret; + do { + ret = dis.read(buf, readLen, buf.length - readLen); + if (ret > 0) { + readLen += ret; + } + } while (ret >= 0); + + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + for (int i = 0; i < writeBytes; i++) { + Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]); + } + } + + // stateful read with ByteBuffer + try (DFSStripedInputStream dis = + new DFSStripedInputStream(fs.getClient(), src, true)) { + ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100); + int readLen = 0; + int ret; + do { + ret = dis.read(buf); + if (ret > 0) { + readLen += ret; + } + } while (ret >= 0); + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + for (int i = 0; i < writeBytes; i++) { + Assert.assertEquals("Byte at i should be the same", getByte(i), buf.array()[i]); + } + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java index b0631cedb6d9b..d980bd6959ad4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java @@ -20,6 +20,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -28,7 +30,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; @@ -38,6 +39,7 @@ import org.junit.Test; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; @@ -52,19 +54,21 @@ public class TestReadStripedFile { private Path filePath = new Path(dirPath, "file"); private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; - private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final int NUM_STRIPE_PER_BLOCK = 2; - private final int BLOCKSIZE = NUM_STRIPE_PER_BLOCK * DATA_BLK_NUM * CELLSIZE; + private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; + private final int BLOCK_GROUP_SIZE = DATA_BLK_NUM * INTERNAL_BLOCK_SIZE; @Before public void setup() throws IOException { - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE); SimulatedFSDataset.setFactory(conf); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(BLK_GROUP_SIZE) - .build(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes( + DATA_BLK_NUM + PARITY_BLK_NUM).build(); cluster.waitActive(); fs = cluster.getFileSystem(); + fs.mkdirs(dirPath); + fs.getClient().createErasureCodingZone(dirPath.toString(), null); } @After @@ -80,10 +84,10 @@ public void tearDown() { @Test public void testGetBlock() throws Exception { final int numBlocks = 4; - DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks, - NUM_STRIPE_PER_BLOCK, true); + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, BLOCKSIZE * numBlocks); + filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); @@ -103,11 +107,11 @@ public void testGetBlock() throws Exception { @Test public void testPread() throws Exception { - final int numBlocks = 4; - DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks, - NUM_STRIPE_PER_BLOCK, true); + final int numBlocks = 2; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, BLOCKSIZE); + filePath.toString(), 0, BLOCK_GROUP_SIZE); assert lbs.get(0) instanceof LocatedStripedBlock; LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); @@ -121,11 +125,89 @@ public void testPread() throws Exception { } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); - int readSize = BLOCKSIZE; + int readSize = BLOCK_GROUP_SIZE; byte[] readBuffer = new byte[readSize]; int ret = in.read(0, readBuffer, 0, readSize); assertEquals(readSize, ret); // TODO: verify read results with patterned data from HDFS-8117 } + + @Test + public void testStatefulRead() throws Exception { + testStatefulRead(false, false); + testStatefulRead(true, false); + testStatefulRead(true, true); + } + + private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) + throws Exception { + final int numBlocks = 2; + final int fileSize = numBlocks * BLOCK_GROUP_SIZE; + if (cellMisalignPacket) { + conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1); + tearDown(); + setup(); + } + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, fileSize); + + assert lbs.getLocatedBlocks().size() == numBlocks; + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + assert lb instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lb); + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); + } + } + + DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), filePath.toString(), + false); + + byte[] expected = new byte[fileSize]; + + for (LocatedBlock bg : lbs.getLocatedBlocks()) { + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = (int) bg.getStartOffset() + + i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[posInFile] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } + } + } + } + + if (useByteBuffer) { + ByteBuffer readBuffer = ByteBuffer.allocate(fileSize); + int done = 0; + while (done < fileSize) { + int ret = in.read(readBuffer); + assertTrue(ret > 0); + done += ret; + } + assertArrayEquals(expected, readBuffer.array()); + } else { + byte[] readBuffer = new byte[fileSize]; + int done = 0; + while (done < fileSize) { + int ret = in.read(readBuffer, done, fileSize - done); + assertTrue(ret > 0); + done += ret; + } + assertArrayEquals(expected, readBuffer); + } + fs.delete(filePath, true); + } } From f9eb95c31da9199b393d7994cf9f21250abe41b9 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 28 Apr 2015 14:14:33 +0530 Subject: [PATCH 085/212] HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY commands from standbynode if any (Contributed by Vinayakumar B) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../org/apache/hadoop/hdfs/server/datanode/BPOfferService.java | 1 + 2 files changed, 4 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index e8db485ef8e69..c28473bae7018 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -134,3 +134,6 @@ HDFS-8033. Erasure coding: stateful (non-positional) read from files in striped layout (Zhe Zhang) + + HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY + commands from standbynode if any (vinayakumarb) \ No newline at end of file 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 69baac7ec3681..6606d0bbc7fde 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 @@ -757,6 +757,7 @@ private boolean processCommandFromStandby(DatanodeCommand cmd, case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE: case DatanodeProtocol.DNA_CACHE: case DatanodeProtocol.DNA_UNCACHE: + case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY: LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction()); break; default: From 5eb17e57265655b9c704e7fe664a29071f7902ae Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 28 Apr 2015 14:24:17 +0530 Subject: [PATCH 086/212] HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated as Idempotent (Contributed by Vinayakumar B) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 ++++- .../hadoop/hdfs/protocol/ClientProtocol.java | 16 ++++++++-------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index c28473bae7018..6c5d7ce86c4eb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -136,4 +136,7 @@ striped layout (Zhe Zhang) HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY - commands from standbynode if any (vinayakumarb) \ No newline at end of file + commands from standbynode if any (vinayakumarb) + + HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated + as Idempotent (vinayakumarb) \ No newline at end of file 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 bba7697bbe56e..76e2d12d9ba3c 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 @@ -1363,14 +1363,6 @@ public EncryptionZone getEZForPath(String src) public BatchedEntries listEncryptionZones( long prevId) throws IOException; - /** - * Create an erasure coding zone with specified schema, if any, otherwise - * default - */ - @Idempotent - public void createErasureCodingZone(String src, ECSchema schema) - throws IOException; - /** * Set xattr of a file or directory. * The name must be prefixed with the namespace followed by ".". For example, @@ -1466,6 +1458,14 @@ public List listXAttrs(String src) @Idempotent public EventBatchList getEditsFromTxid(long txid) throws IOException; + /** + * Create an erasure coding zone with specified schema, if any, otherwise + * default + */ + @AtMostOnce + public void createErasureCodingZone(String src, ECSchema schema) + throws IOException; + /** * Gets the ECInfo for the specified file/directory * From 9593776e34d0de956798f1fd0092d9ef4e67d005 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 28 Apr 2015 13:42:24 -0700 Subject: [PATCH 087/212] HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open. Contributed by Kai Sasaki. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 ++++- .../java/org/apache/hadoop/hdfs/DFSClient.java | 7 ++++++- .../hadoop/hdfs/DFSStripedInputStream.java | 5 +++-- .../hadoop/hdfs/TestDFSStripedInputStream.java | 16 +++++++--------- .../apache/hadoop/hdfs/TestReadStripedFile.java | 11 ++++++++--- 5 files changed, 28 insertions(+), 16 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 6c5d7ce86c4eb..9b4bf24a725cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -139,4 +139,7 @@ commands from standbynode if any (vinayakumarb) HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated - as Idempotent (vinayakumarb) \ No newline at end of file + as Idempotent (vinayakumarb) + + HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open. + (Kai Sasaki via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 0a67cf7550e05..722bb39695d13 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1193,7 +1193,12 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) // Get block info from namenode TraceScope scope = getPathTraceScope("newDFSInputStream", src); try { - return new DFSInputStream(this, src, verifyChecksum); + ECInfo info = getErasureCodingInfo(src); + if (info != null) { + return new DFSStripedInputStream(this, src, verifyChecksum, info); + } else { + return new DFSInputStream(this, src, verifyChecksum); + } } finally { scope.close(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index fe9e101c26f2e..f6f7ed21bdb47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -134,11 +134,12 @@ static ReadPortion[] planReadPortions(final int dataBlkNum, private final short parityBlkNum; private final ECInfo ecInfo; - DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum) + DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECInfo info) throws IOException { super(dfsClient, src, verifyChecksum); // ECInfo is restored from NN just before reading striped file. - ecInfo = dfsClient.getErasureCodingInfo(src); + assert info != null; + ecInfo = info; cellSize = ecInfo.getSchema().getChunkSize(); dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits(); parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index cf109818c3369..bcfc74b3d5bbf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -167,10 +168,9 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) writeBytes, fileLength); // pread - try (DFSStripedInputStream dis = - new DFSStripedInputStream(fs.getClient(), src, true)) { + try (FSDataInputStream fsdis = fs.open(new Path(src))) { byte[] buf = new byte[writeBytes + 100]; - int readLen = dis.read(0, buf, 0, buf.length); + int readLen = fsdis.read(0, buf, 0, buf.length); readLen = readLen >= 0 ? readLen : 0; Assert.assertEquals("The length of file should be the same to write size", writeBytes, readLen); @@ -180,13 +180,12 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) } // stateful read with byte array - try (DFSStripedInputStream dis = - new DFSStripedInputStream(fs.getClient(), src, true)) { + try (FSDataInputStream fsdis = fs.open(new Path(src))) { byte[] buf = new byte[writeBytes + 100]; int readLen = 0; int ret; do { - ret = dis.read(buf, readLen, buf.length - readLen); + ret = fsdis.read(buf, readLen, buf.length - readLen); if (ret > 0) { readLen += ret; } @@ -201,13 +200,12 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) } // stateful read with ByteBuffer - try (DFSStripedInputStream dis = - new DFSStripedInputStream(fs.getClient(), src, true)) { + try (FSDataInputStream fsdis = fs.open(new Path(src))) { ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100); int readLen = 0; int ret; do { - ret = dis.read(buf); + ret = fsdis.read(buf); if (ret > 0) { readLen += ret; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java index d980bd6959ad4..1ad480e7f2952 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java @@ -24,6 +24,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; @@ -33,6 +34,7 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; import org.junit.Before; @@ -52,6 +54,8 @@ public class TestReadStripedFile { private DistributedFileSystem fs; private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); + private ECInfo info = new ECInfo(filePath.toString(), + ECSchemaManager.getSystemDefaultSchema()); private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @@ -89,7 +93,7 @@ public void testGetBlock() throws Exception { LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); final DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); + new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info); List lbList = lbs.getLocatedBlocks(); for (LocatedBlock aLbList : lbList) { @@ -124,7 +128,8 @@ public void testPread() throws Exception { bg.getBlock().getBlockPoolId()); } DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), filePath.toString(), false); + new DFSStripedInputStream(fs.getClient(), + filePath.toString(), false, info); int readSize = BLOCK_GROUP_SIZE; byte[] readBuffer = new byte[readSize]; int ret = in.read(0, readBuffer, 0, readSize); @@ -170,7 +175,7 @@ private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), - false); + false, info); byte[] expected = new byte[fileSize]; From 1a8139e6addf7df6459b397c813ba3bf2c74e3c2 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 29 Apr 2015 11:35:58 -0700 Subject: [PATCH 088/212] Fix merge conflicts. --- .../org/apache/hadoop/hdfs/DFSInputStream.java | 7 +++---- .../hadoop/hdfs/DFSStripedOutputStream.java | 15 ++++----------- .../apache/hadoop/hdfs/StripedDataStreamer.java | 7 ++++--- 3 files changed, 11 insertions(+), 18 deletions(-) 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 6649f4c87ca1d..6e58cd6001909 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 @@ -1119,7 +1119,7 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, /** * Read data from one DataNode. * @param datanode the datanode from which to read data - * @param block the block to read + * @param blockStartOffset starting offset in the file * @param startInBlk the startInBlk offset of the block * @param endInBlk the endInBlk offset of the block * @param buf the given byte array into which the data is read @@ -1149,7 +1149,7 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, BlockReader reader = null; try { DFSClientFaultInjector.get().fetchFromDatanodeException(); - reader = getBlockReader(block, start, len, datanode.addr, + reader = getBlockReader(block, startInBlk, len, datanode.addr, datanode.storageType, datanode.info); for (int i = 0; i < offsets.length; i++) { int nread = reader.readAll(buf, offsets[i], lengths[i]); @@ -1206,8 +1206,7 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, * with each other. */ private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) { - Preconditions.checkArgument(offsets.length == lengths.length && - offsets.length > 0); + Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0); int sum = 0; for (int i = 0; i < lengths.length; i++) { if (i > 0) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 68422675a1ec6..c9301877b8dc0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -124,10 +124,7 @@ private long getBlockGroupSize() { for (short i = 0; i < numAllBlocks; i++) { StripedDataStreamer streamer = new StripedDataStreamer(stat, null, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager, - i, stripeBlocks); - if (favoredNodes != null && favoredNodes.length != 0) { - streamer.setFavoredNodes(favoredNodes); - } + i, stripeBlocks, favoredNodes); s.add(streamer); } streamers = Collections.unmodifiableList(s); @@ -316,7 +313,7 @@ synchronized void abort() throws IOException { return; } for (StripedDataStreamer streamer : streamers) { - streamer.setLastException(new IOException("Lease timeout of " + streamer.getLastException().set(new IOException("Lease timeout of " + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired.")); } @@ -414,12 +411,8 @@ void setClosed() { @Override protected synchronized void closeImpl() throws IOException { if (isClosed()) { - IOException e = getLeadingStreamer().getLastException().getAndSet(null); - if (e != null) { - throw e; - } else { - return; - } + getLeadingStreamer().getLastException().check(); + return; } try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 19c205e27b8ab..ef7e2a687a6b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -58,9 +58,10 @@ public class StripedDataStreamer extends DataStreamer { Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage, short index, - List> stripedBlocks) { - super(stat,block, dfsClient, src, progress, checksum, cachingStrategy, - byteArrayManage); + List> stripedBlocks, + String[] favoredNodes) { + super(stat, block, dfsClient, src, progress, checksum, cachingStrategy, + byteArrayManage, favoredNodes); this.index = index; this.stripedBlocks = stripedBlocks; } From b00c66387709bd7c2411eb9778b5d15de53672fc Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 29 Apr 2015 15:53:31 -0700 Subject: [PATCH 089/212] HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream (stateful read). Contributed by Jing Zhao --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 336 ++++++++---------- 2 files changed, 150 insertions(+), 189 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9b4bf24a725cf..6a9bdee972ca4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -143,3 +143,6 @@ HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open. (Kai Sasaki via jing9) + + HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream + (stateful read). (Jing Zhao via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index f6f7ed21bdb47..3da7306f4db56 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -22,11 +22,8 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; -import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; -import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.token.Token; import org.apache.htrace.Span; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; @@ -126,23 +123,42 @@ static ReadPortion[] planReadPortions(final int dataBlkNum, return results; } + private static class ReaderRetryPolicy { + private int fetchEncryptionKeyTimes = 1; + private int fetchTokenTimes = 1; + + void refetchEncryptionKey() { + fetchEncryptionKeyTimes--; + } + + void refetchToken() { + fetchTokenTimes--; + } + + boolean shouldRefetchEncryptionKey() { + return fetchEncryptionKeyTimes > 0; + } + + boolean shouldRefetchToken() { + return fetchTokenTimes > 0; + } + } + private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS; - private BlockReader[] blockReaders = null; - private DatanodeInfo[] currentNodes = null; + private final BlockReader[] blockReaders = new BlockReader[groupSize]; + private final DatanodeInfo[] currentNodes = new DatanodeInfo[groupSize]; private final int cellSize; private final short dataBlkNum; private final short parityBlkNum; - private final ECInfo ecInfo; - DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECInfo info) - throws IOException { + DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, + ECInfo ecInfo) throws IOException { super(dfsClient, src, verifyChecksum); // ECInfo is restored from NN just before reading striped file. - assert info != null; - ecInfo = info; + assert ecInfo != null; cellSize = ecInfo.getSchema().getChunkSize(); - dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits(); - parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits(); + dataBlkNum = (short) ecInfo.getSchema().getNumDataUnits(); + parityBlkNum = (short) ecInfo.getSchema().getNumParityUnits(); DFSClient.LOG.debug("Creating an striped input stream for file " + src); } @@ -162,9 +178,7 @@ public synchronized int read(final ByteBuffer buf) throws IOException { * When seeking into a new block group, create blockReader for each internal * block in the group. */ - @VisibleForTesting - private synchronized DatanodeInfo[] blockSeekTo(long target) - throws IOException { + private synchronized void blockSeekTo(long target) throws IOException { if (target >= getFileLength()) { throw new IOException("Attempted to read past end of file"); } @@ -172,18 +186,13 @@ private synchronized DatanodeInfo[] blockSeekTo(long target) // Will be getting a new BlockReader. closeCurrentBlockReaders(); - // Connect to best DataNode for desired Block, with potential offset - DatanodeInfo[] chosenNodes = new DatanodeInfo[groupSize]; - int refetchToken = 1; // only need to get a new access token once - int refetchEncryptionKey = 1; // only need to get a new encryption key once - // Compute desired striped block group LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target); - // Update current position this.pos = target; this.blockEnd = targetBlockGroup.getStartOffset() + targetBlockGroup.getBlockSize() - 1; + currentLocatedBlock = targetBlockGroup; long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset(); LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup( @@ -191,71 +200,50 @@ private synchronized DatanodeInfo[] blockSeekTo(long target) // The purpose is to get start offset into each block ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, offsetIntoBlockGroup, 0, 0); + + final ReaderRetryPolicy retry = new ReaderRetryPolicy(); + for (int i = 0; i < groupSize; i++) { + LocatedBlock targetBlock = targetBlocks[i]; + if (targetBlock != null) { + DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); + if (retval != null) { + currentNodes[i] = retval.info; + blockReaders[i] = getBlockReaderWithRetry(targetBlock, + readPortions[i].startOffsetInBlock, + targetBlock.getBlockSize() - readPortions[i].startOffsetInBlock, + retval.addr, retval.storageType, retval.info, target, retry); + } + } + } + } + + private BlockReader getBlockReaderWithRetry(LocatedBlock targetBlock, + long offsetInBlock, long length, InetSocketAddress targetAddr, + StorageType storageType, DatanodeInfo datanode, long offsetInFile, + ReaderRetryPolicy retry) throws IOException { + // only need to get a new access token or a new encryption key once while (true) { - int i = 0; - InetSocketAddress targetAddr = null; try { - blockReaders = new BlockReader[groupSize]; - for (i = 0; i < groupSize; i++) { - LocatedBlock targetBlock = targetBlocks[i]; - if (targetBlock == null) { - continue; - } - long offsetIntoBlock = readPortions[i].startOffsetInBlock; - DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); - chosenNodes[i] = retval.info; - targetAddr = retval.addr; - StorageType storageType = retval.storageType; - - ExtendedBlock blk = targetBlock.getBlock(); - Token accessToken = targetBlock.getBlockToken(); - CachingStrategy curCachingStrategy; - boolean shortCircuitForbidden; - synchronized(infoLock) { - curCachingStrategy = cachingStrategy; - shortCircuitForbidden = shortCircuitForbidden(); - } - blockReaders[i] = new BlockReaderFactory(dfsClient.getConf()). - setInetSocketAddress(targetAddr). - setRemotePeerFactory(dfsClient). - setDatanodeInfo(chosenNodes[i]). - setStorageType(storageType). - setFileName(src). - setBlock(blk). - setBlockToken(accessToken). - setStartOffset(offsetIntoBlock). - setVerifyChecksum(verifyChecksum). - setClientName(dfsClient.clientName). - setLength(blk.getNumBytes() - offsetIntoBlock). - setCachingStrategy(curCachingStrategy). - setAllowShortCircuitLocalReads(!shortCircuitForbidden). - setClientCacheContext(dfsClient.getClientContext()). - setUserGroupInformation(dfsClient.ugi). - setConfiguration(dfsClient.getConfiguration()). - build(); - } - currentLocatedBlock = targetBlockGroup; - return chosenNodes; - } catch (IOException ex) { - // Retry in case of encryption key or token exceptions. Otherwise throw - // IOException: since each internal block is singly replicated, it's - // not meaningful trying to locate another replica. - if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { + return getBlockReader(targetBlock, offsetInBlock, length, targetAddr, + storageType, datanode); + } catch (IOException e) { + if (e instanceof InvalidEncryptionKeyException && + retry.shouldRefetchEncryptionKey()) { DFSClient.LOG.info("Will fetch a new encryption key and retry, " + "encryption key was invalid when connecting to " + targetAddr - + " : " + ex); - // The encryption key used is invalid. - refetchEncryptionKey--; + + " : " + e); dfsClient.clearDataEncryptionKey(); - } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) { - refetchToken--; - fetchBlockAt(target); + retry.refetchEncryptionKey(); + } else if (retry.shouldRefetchToken() && + tokenRefetchNeeded(e, targetAddr)) { + fetchBlockAt(offsetInFile); + retry.refetchToken(); } else { DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block" - + ", add to deadNodes and continue. " + ex, ex); - // Put chosen node into dead list and throw exception - addToDeadNodes(chosenNodes[i]); - throw ex; + + ", add to deadNodes and continue.", e); + // Put chosen node into dead list, continue + addToDeadNodes(datanode); + return null; } } } @@ -272,15 +260,15 @@ protected void closeCurrentBlockReaders() { return; } for (int i = 0; i < groupSize; i++) { - if (blockReaders[i] == null) { - continue; - } - try { - blockReaders[i].close(); - } catch (IOException e) { - DFSClient.LOG.error("error closing blockReader", e); + if (blockReaders[i] != null) { + try { + blockReaders[i].close(); + } catch (IOException e) { + DFSClient.LOG.error("error closing blockReader", e); + } + blockReaders[i] = null; } - blockReaders[i] = null; + currentNodes[i] = null; } blockEnd = -1; } @@ -292,123 +280,93 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, if (closed.get()) { throw new IOException("Stream closed"); } - Map> corruptedBlockMap - = new HashMap<>(); + Map> corruptedBlockMap = new HashMap<>(); failures = 0; if (pos < getFileLength()) { - int retries = 2; /** Index of the target block in a stripe to read from */ int idxInGroup = (int) ((pos / cellSize) % dataBlkNum); - while (retries > 0) { - try { - // currentNode can be left as null if previous read had a checksum - // error on the same block. See HDFS-3067 - if (pos > blockEnd || currentNodes == null) { - currentNodes = blockSeekTo(pos); - } - int realLen = (int) Math.min(len, (blockEnd - pos + 1L)); - synchronized(infoLock) { - if (locatedBlocks.isLastBlockComplete()) { - realLen = (int) Math.min(realLen, - locatedBlocks.getFileLength() - pos); - } + try { + if (pos > blockEnd) { + blockSeekTo(pos); + } + int realLen = (int) Math.min(len, (blockEnd - pos + 1L)); + synchronized (infoLock) { + if (locatedBlocks.isLastBlockComplete()) { + realLen = (int) Math.min(realLen, + locatedBlocks.getFileLength() - pos); } + } - /** Number of bytes already read into buffer */ - int result = 0; - while (result < realLen) { - /** - * Temporary position into the file; {@link pos} might not proceed - * to this temporary position in case of exceptions. - */ - long tmpPos = pos + result; - /** Start and end offsets of a cell in the file */ - long cellStart = (tmpPos / cellSize) * cellSize; - long cellEnd = cellStart + cellSize - 1; - - /** Number of bytes to read from the current cell */ - int realLenInCell = (int) Math.min(realLen - result, - cellEnd - tmpPos + 1L); - assert realLenInCell > 0 : "Temporary position shouldn't be " + - "after cellEnd"; - // Read from one blockReader up to cell boundary - int cellRet = readBuffer(blockReaders[idxInGroup], - currentNodes[idxInGroup], strategy, off + result, - realLenInCell); - if (cellRet >= 0) { - result += cellRet; - if (cellRet < realLenInCell) { - // A short read indicates the current blockReader buffer is - // already drained. Should return the read call. Otherwise - // should proceed to the next cell. - break; - } - } else { - // got a EOS from reader though we expect more data on it. - throw new IOException("Unexpected EOS from the reader"); + /** Number of bytes already read into buffer */ + int result = 0; + while (result < realLen) { + /** + * Temporary position into the file; {@link pos} might not proceed + * to this temporary position in case of exceptions. + */ + long tmpPos = pos + result; + /** Start and end offsets of a cell in the file */ + long cellStart = (tmpPos / cellSize) * cellSize; + long cellEnd = cellStart + cellSize - 1; + + /** Number of bytes to read from the current cell */ + int realLenInCell = (int) Math.min(realLen - result, + cellEnd - tmpPos + 1L); + assert realLenInCell > 0 : "Temporary position shouldn't be " + + "after cellEnd"; + + // Read from one blockReader up to cell boundary + int cellRet = readBuffer(blockReaders[idxInGroup], + currentNodes[idxInGroup], strategy, off + result, realLenInCell, + corruptedBlockMap); + if (cellRet >= 0) { + result += cellRet; + if (cellRet < realLenInCell) { + // A short read indicates the current blockReader buffer is + // already drained. Should return the read call. Otherwise + // should proceed to the next cell. + break; } - idxInGroup = (idxInGroup + 1) % dataBlkNum; - } - - pos += result; - - if (dfsClient.stats != null) { - dfsClient.stats.incrementBytesRead(result); - } - return result; - } catch (ChecksumException ce) { - throw ce; - } catch (IOException e) { - if (retries == 1) { - DFSClient.LOG.warn("DFS Read", e); - } - blockEnd = -1; - if (currentNodes[idxInGroup] != null) { - addToDeadNodes(currentNodes[idxInGroup]); + } else { + // got a EOS from reader though we expect more data on it. + throw new IOException("Unexpected EOS from the reader"); } - if (--retries == 0) { - throw e; - } - } finally { - // Check if need to report block replicas corruption either read - // was successful or ChecksumException occured. - reportCheckSumFailure(corruptedBlockMap, - currentLocatedBlock.getLocations().length); + idxInGroup = (idxInGroup + 1) % dataBlkNum; } + pos += result; + if (dfsClient.stats != null) { + dfsClient.stats.incrementBytesRead(result); + } + return result; + } finally { + // Check if need to report block replicas corruption either read + // was successful or ChecksumException occured. + reportCheckSumFailure(corruptedBlockMap, + currentLocatedBlock.getLocations().length); } } return -1; } private synchronized int readBuffer(BlockReader blockReader, - DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len) - throws IOException { - IOException ioe; - while (true) { - try { - return readerStrategy.doRead(blockReader, off, len); - } catch ( ChecksumException ce ) { - DFSClient.LOG.warn("Found Checksum error for " - + getCurrentBlock() + " from " + currentNode - + " at " + ce.getPos()); - // If current block group is corrupt, it's meaningless to retry. - // TODO: this should trigger decoding logic (HDFS-7678) - throw ce; - } catch ( IOException e ) { - ioe = e; - } - - boolean sourceFound = seekToBlockSource(pos); - if (!sourceFound) { - throw ioe; - } + DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len, + Map> corruptedBlockMap) { + try { + return readerStrategy.doRead(blockReader, off, len); + } catch ( ChecksumException ce ) { + DFSClient.LOG.warn("Found Checksum error for " + + getCurrentBlock() + " from " + currentNode + + " at " + ce.getPos()); + // we want to remember which block replicas we have tried + addIntoCorruptedBlockMap(getCurrentBlock(), currentNode, + corruptedBlockMap); + } catch (IOException e) { + DFSClient.LOG.warn("Exception while reading from " + + getCurrentBlock() + " of " + src + " from " + + currentNode, e); } - } - - private boolean seekToBlockSource(long targetPos) - throws IOException { - currentNodes = blockSeekTo(targetPos); - return true; + // TODO: this should trigger decoding logic (HDFS-7678) + return -1; } protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy { @@ -418,7 +376,7 @@ protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy { @Override public int doRead(BlockReader blockReader, int off, int len) - throws ChecksumException, IOException { + throws IOException { int oldlimit = buf.limit(); if (buf.remaining() > len) { buf.limit(buf.position() + len); From f0628280c351640a807b05eaf786d051b5a7bd66 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 29 Apr 2015 23:49:52 -0700 Subject: [PATCH 090/212] HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil. Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 111 +---------- .../hadoop/hdfs/util/StripedBlockUtil.java | 174 ++++++++++++++++++ .../hadoop/hdfs/TestPlanReadPortions.java | 11 +- 4 files changed, 186 insertions(+), 113 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 6a9bdee972ca4..ca60487ecc972 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -146,3 +146,6 @@ HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream (stateful read). (Jing Zhao via Zhe Zhang) + + HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil. + (Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 3da7306f4db56..0dc98fdbb00a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.hdfs; -import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions; + import org.apache.hadoop.net.NetUtils; import org.apache.htrace.Span; import org.apache.htrace.Trace; @@ -31,8 +33,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; import java.util.Set; import java.util.Map; import java.util.HashMap; @@ -69,59 +69,6 @@ * 3. pread with decode support: TODO: will be supported after HDFS-7678 *****************************************************************************/ public class DFSStripedInputStream extends DFSInputStream { - /** - * This method plans the read portion from each block in the stripe - * @param dataBlkNum The number of data blocks in the striping group - * @param cellSize The size of each striping cell - * @param startInBlk Starting offset in the striped block - * @param len Length of the read request - * @param bufOffset Initial offset in the result buffer - * @return array of {@link ReadPortion}, each representing the portion of I/O - * for an individual block in the group - */ - @VisibleForTesting - static ReadPortion[] planReadPortions(final int dataBlkNum, - final int cellSize, final long startInBlk, final int len, int bufOffset) { - ReadPortion[] results = new ReadPortion[dataBlkNum]; - for (int i = 0; i < dataBlkNum; i++) { - results[i] = new ReadPortion(); - } - - // cellIdxInBlk is the index of the cell in the block - // E.g., cell_3 is the 2nd cell in blk_0 - int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum)); - - // blkIdxInGroup is the index of the block in the striped block group - // E.g., blk_2 is the 3rd block in the group - final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum); - results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk + - startInBlk % cellSize; - boolean crossStripe = false; - for (int i = 1; i < dataBlkNum; i++) { - if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) { - cellIdxInBlk++; - crossStripe = true; - } - results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock = - cellSize * cellIdxInBlk; - } - - int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len); - results[blkIdxInGroup].offsetsInBuf.add(bufOffset); - results[blkIdxInGroup].lengths.add(firstCellLen); - results[blkIdxInGroup].readLength += firstCellLen; - - int i = (blkIdxInGroup + 1) % dataBlkNum; - for (int done = firstCellLen; done < len; done += cellSize) { - ReadPortion rp = results[i]; - rp.offsetsInBuf.add(done + bufOffset); - final int readLen = Math.min(len - done, cellSize); - rp.lengths.add(readLen); - rp.readLength += readLen; - i = (i + 1) % dataBlkNum; - } - return results; - } private static class ReaderRetryPolicy { private int fetchEncryptionKeyTimes = 1; @@ -520,56 +467,4 @@ private void waitNextCompletion(CompletionService stripedReadsService, } throw new InterruptedException("let's retry"); } - - - /** - * This class represents the portion of I/O associated with each block in the - * striped block group. - */ - static class ReadPortion { - /** - * startOffsetInBlock - * | - * v - * |<-lengths[0]->|<- lengths[1] ->|<-lengths[2]->| - * +------------------+------------------+----------------+ - * | cell_0 | cell_3 | cell_6 | <- blk_0 - * +------------------+------------------+----------------+ - * _/ \_______________________ - * | | - * v offsetsInBuf[0] v offsetsInBuf[1] - * +------------------------------------------------------+ - * | cell_0 | cell_1 and cell_2 |cell_3 ...| <- buf - * | (partial) | (from blk_1 and blk_2) | | - * +------------------------------------------------------+ - */ - private long startOffsetInBlock = 0; - private int readLength = 0; - private final List offsetsInBuf = new ArrayList<>(); - private final List lengths = new ArrayList<>(); - - int[] getOffsets() { - int[] offsets = new int[offsetsInBuf.size()]; - for (int i = 0; i < offsets.length; i++) { - offsets[i] = offsetsInBuf.get(i); - } - return offsets; - } - - int[] getLengths() { - int[] lens = new int[this.lengths.size()]; - for (int i = 0; i < lens.length; i++) { - lens[i] = this.lengths.get(i); - } - return lens; - } - - int getReadLength() { - return readLength; - } - - long getStartOffsetInBlock() { - return startOffsetInBlock; - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index d622d4dab0766..cb6d39a692452 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.util; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -27,6 +28,15 @@ import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + /** * Utility class for analyzing striped block groups */ @@ -134,4 +144,168 @@ public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum, + offsetInBlk % cellSize; // partial cell } + /** + * This method plans the read portion from each block in the stripe + * @param dataBlkNum The number of data blocks in the striping group + * @param cellSize The size of each striping cell + * @param startInBlk Starting offset in the striped block + * @param len Length of the read request + * @param bufOffset Initial offset in the result buffer + * @return array of {@link ReadPortion}, each representing the portion of I/O + * for an individual block in the group + */ + @VisibleForTesting + public static ReadPortion[] planReadPortions(final int dataBlkNum, + final int cellSize, final long startInBlk, final int len, int bufOffset) { + ReadPortion[] results = new ReadPortion[dataBlkNum]; + for (int i = 0; i < dataBlkNum; i++) { + results[i] = new ReadPortion(); + } + + // cellIdxInBlk is the index of the cell in the block + // E.g., cell_3 is the 2nd cell in blk_0 + int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum)); + + // blkIdxInGroup is the index of the block in the striped block group + // E.g., blk_2 is the 3rd block in the group + final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum); + results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk + + startInBlk % cellSize; + boolean crossStripe = false; + for (int i = 1; i < dataBlkNum; i++) { + if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) { + cellIdxInBlk++; + crossStripe = true; + } + results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock = + cellSize * cellIdxInBlk; + } + + int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len); + results[blkIdxInGroup].offsetsInBuf.add(bufOffset); + results[blkIdxInGroup].lengths.add(firstCellLen); + results[blkIdxInGroup].readLength += firstCellLen; + + int i = (blkIdxInGroup + 1) % dataBlkNum; + for (int done = firstCellLen; done < len; done += cellSize) { + ReadPortion rp = results[i]; + rp.offsetsInBuf.add(done + bufOffset); + final int readLen = Math.min(len - done, cellSize); + rp.lengths.add(readLen); + rp.readLength += readLen; + i = (i + 1) % dataBlkNum; + } + return results; + } + + /** + * Get the next completed striped read task + * + * @return {@link StripedReadResult} indicating the status of the read task + * succeeded, and the block index of the task. If the method times + * out without getting any completed read tasks, -1 is returned as + * block index. + * @throws InterruptedException + */ + public static StripedReadResult getNextCompletedStripedRead( + CompletionService readService, Map, + Integer> futures, final long threshold) throws InterruptedException { + Preconditions.checkArgument(!futures.isEmpty()); + Preconditions.checkArgument(threshold > 0); + Future future = null; + try { + future = readService.poll(threshold, TimeUnit.MILLISECONDS); + if (future != null) { + future.get(); + return new StripedReadResult(futures.remove(future), + StripedReadResult.SUCCESSFUL); + } else { + return new StripedReadResult(StripedReadResult.TIMEOUT); + } + } catch (ExecutionException e) { + return new StripedReadResult(futures.remove(future), + StripedReadResult.FAILED); + } catch (CancellationException e) { + return new StripedReadResult(futures.remove(future), + StripedReadResult.CANCELLED); + } + } + + /** + * This class represents the portion of I/O associated with each block in the + * striped block group. + */ + public static class ReadPortion { + /** + * startOffsetInBlock + * | + * v + * |<-lengths[0]->|<- lengths[1] ->|<-lengths[2]->| + * +------------------+------------------+----------------+ + * | cell_0 | cell_3 | cell_6 | <- blk_0 + * +------------------+------------------+----------------+ + * _/ \_______________________ + * | | + * v offsetsInBuf[0] v offsetsInBuf[1] + * +------------------------------------------------------+ + * | cell_0 | cell_1 and cell_2 |cell_3 ...| <- buf + * | (partial) | (from blk_1 and blk_2) | | + * +------------------------------------------------------+ + */ + public long startOffsetInBlock = 0; + public int readLength = 0; + public final List offsetsInBuf = new ArrayList<>(); + public final List lengths = new ArrayList<>(); + + public int[] getOffsets() { + int[] offsets = new int[offsetsInBuf.size()]; + for (int i = 0; i < offsets.length; i++) { + offsets[i] = offsetsInBuf.get(i); + } + return offsets; + } + + public int[] getLengths() { + int[] lens = new int[this.lengths.size()]; + for (int i = 0; i < lens.length; i++) { + lens[i] = this.lengths.get(i); + } + return lens; + } + + public boolean containsReadPortion(ReadPortion rp) { + long end = startOffsetInBlock + readLength; + return startOffsetInBlock <= rp.startOffsetInBlock && end >= + rp.startOffsetInBlock + rp.readLength; + } + } + + /** + * This class represents result from a striped read request. + * If the task was successful or the internal computation failed, + * an index is also returned. + */ + public static class StripedReadResult { + public static final int SUCCESSFUL = 0x01; + public static final int FAILED = 0x02; + public static final int TIMEOUT = 0x04; + public static final int CANCELLED = 0x08; + + public final int index; + public final int state; + + public StripedReadResult(int state) { + Preconditions.checkArgument(state == TIMEOUT, + "Only timeout result should return negative index."); + this.index = -1; + this.state = state; + } + + public StripedReadResult(int index, int state) { + Preconditions.checkArgument(state != TIMEOUT, + "Timeout result should return negative index."); + this.index = index; + this.state = state; + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java index cf84b30340a94..3b5787a4a94f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java @@ -19,7 +19,8 @@ import org.junit.Test; -import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; import static org.junit.Assert.*; public class TestPlanReadPortions { @@ -32,13 +33,13 @@ public class TestPlanReadPortions { private void testPlanReadPortions(int startInBlk, int length, int bufferOffset, int[] readLengths, int[] offsetsInBlock, int[][] bufferOffsets, int[][] bufferLengths) { - ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE, + ReadPortion[] results = StripedBlockUtil.planReadPortions(GROUP_SIZE, CELLSIZE, startInBlk, length, bufferOffset); assertEquals(GROUP_SIZE, results.length); for (int i = 0; i < GROUP_SIZE; i++) { - assertEquals(readLengths[i], results[i].getReadLength()); - assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock()); + assertEquals(readLengths[i], results[i].readLength); + assertEquals(offsetsInBlock[i], results[i].startOffsetInBlock); final int[] bOffsets = results[i].getOffsets(); assertArrayEquals(bufferOffsets[i], bOffsets); final int[] bLengths = results[i].getLengths(); @@ -47,7 +48,7 @@ private void testPlanReadPortions(int startInBlk, int length, } /** - * Test {@link DFSStripedInputStream#planReadPortions} + * Test {@link StripedBlockUtil#planReadPortions} */ @Test public void testPlanReadPortions() { From 1a31f1c303e8f541f0b22eccf85f3b789a220f21 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 30 Apr 2015 00:13:32 -0700 Subject: [PATCH 091/212] HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of datastreamer threads. Contributed by Rakesh R. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../apache/hadoop/hdfs/DFSStripedOutputStream.java | 12 ++++++++++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index ca60487ecc972..3c751520d7bf8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -149,3 +149,6 @@ HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil. (Zhe Zhang) + + HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of + datastreamer threads. (Rakesh R via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index c9301877b8dc0..5e2a534ed78a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -331,18 +331,26 @@ boolean isClosed() { // interrupt datastreamer if force is true @Override protected void closeThreads(boolean force) throws IOException { + int index = 0; + boolean exceptionOccurred = false; for (StripedDataStreamer streamer : streamers) { try { streamer.close(force); streamer.join(); streamer.closeSocket(); - } catch (InterruptedException e) { - throw new IOException("Failed to shutdown streamer"); + } catch (InterruptedException | IOException e) { + DFSClient.LOG.error("Failed to shutdown streamer: name=" + + streamer.getName() + ", index=" + index + ", file=" + src, e); + exceptionOccurred = true; } finally { streamer.setSocketToNull(); setClosed(); + index++; } } + if (exceptionOccurred) { + throw new IOException("Failed to shutdown streamer"); + } } /** From 5059958bf4f5541926bae3b145bceb2fd583e4ed Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 30 Apr 2015 19:42:29 -0700 Subject: [PATCH 092/212] HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() when loading editlog. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../namenode/ErasureCodingZoneManager.java | 3 +- .../hdfs/server/namenode/FSNamesystem.java | 4 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 12 ++++ .../hadoop/hdfs/TestErasureCodingZones.java | 6 +- .../server/namenode/TestAddStripedBlocks.java | 61 ++++++++++--------- 6 files changed, 52 insertions(+), 37 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 3c751520d7bf8..596bbcfdd245d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -152,3 +152,6 @@ HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of datastreamer threads. (Rakesh R via Zhe Zhang) + + HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() + when loading editlog. (jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 8cda289849d74..14d4e29c040e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -79,7 +79,8 @@ ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { for (XAttr xAttr : xAttrs) { if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { String schemaName = new String(xAttr.getValue()); - ECSchema schema = dir.getFSNamesystem().getECSchema(schemaName); + ECSchema schema = dir.getFSNamesystem().getSchemaManager() + .getSchema(schemaName); return new ECZoneInfo(inode.getFullPathName(), schema); } } 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 a2091427873cf..3fa88184a923c 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 @@ -7529,9 +7529,9 @@ BatchedListEntries listEncryptionZones(long prevId) /** * Create an erasure coding zone on directory src. - * @param schema ECSchema for the erasure coding zone - * @param src the path of a directory which will be the root of the + * @param srcArg the path of a directory which will be the root of the * erasure coding zone. The directory must be empty. + * @param schema ECSchema for the erasure coding zone * * @throws AccessControlException if the caller is not the superuser. * @throws UnresolvedLinkException if the path can't be resolved. 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 0c88842f9b517..0165189bc9cac 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 @@ -1948,4 +1948,16 @@ public static Block addStripedBlockToFile(List dataNodes, lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS); return lastBlock; } + + /** + * Because currently DFSStripedOutputStream does not support hflush/hsync, + * tests can use this method to flush all the buffered data to DataNodes. + */ + public static void writeAndFlushStripedOutputStream( + DFSStripedOutputStream out, int chunkSize) throws IOException { + // FSOutputSummer.BUFFER_NUM_CHUNKS == 9 + byte[] toWrite = new byte[chunkSize * 9 + 1]; + out.write(toWrite); + out.flushInternal(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index ae2bdd85fb03d..f1aec821b9ff2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -35,8 +35,6 @@ import static org.junit.Assert.*; public class TestErasureCodingZones { - private final int NUM_OF_DATANODES = 3; - private Configuration conf; private MiniDFSCluster cluster; private DistributedFileSystem fs; private static final int BLOCK_SIZE = 1024; @@ -44,10 +42,10 @@ public class TestErasureCodingZones { @Before public void setupCluster() throws IOException { - conf = new HdfsConfiguration(); + Configuration conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); cluster = new MiniDFSCluster.Builder(conf). - numDataNodes(NUM_OF_DATANODES).build(); + numDataNodes(1).build(); cluster.waitActive(); fs = cluster.getFileSystem(); namesystem = cluster.getNamesystem(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index d03e938b5f1c6..297db14978006 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -19,6 +19,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSStripedOutputStream; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; @@ -55,6 +56,7 @@ import java.util.List; import java.util.UUID; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; import static org.junit.Assert.assertEquals; @@ -103,52 +105,50 @@ public void testAllocateBlockId() throws Exception { Assert.assertEquals(firstId + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId); } - @Test + @Test (timeout=60000) public void testAddStripedBlock() throws Exception { final Path file = new Path("/file1"); // create an empty file FSDataOutputStream out = null; try { out = dfs.create(file, (short) 1); + DFSTestUtil.writeAndFlushStripedOutputStream( + (DFSStripedOutputStream) out.getWrappedStream(), + DFS_BYTES_PER_CHECKSUM_DEFAULT); FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); - LocatedBlock newBlock = cluster.getNamesystem().getAdditionalBlock( - file.toString(), fileNode.getId(), dfs.getClient().getClientName(), - null, null, null); - assertEquals(GROUP_SIZE, newBlock.getLocations().length); - assertEquals(GROUP_SIZE, newBlock.getStorageIDs().length); BlockInfo[] blocks = fileNode.getBlocks(); assertEquals(1, blocks.length); Assert.assertTrue(blocks[0].isStriped()); checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true); + + // restart NameNode to check editlog + cluster.restartNameNode(true); + fsdir = cluster.getNamesystem().getFSDirectory(); + fileNode = fsdir.getINode4Write(file.toString()).asFile(); + blocks = fileNode.getBlocks(); + assertEquals(1, blocks.length); + Assert.assertTrue(blocks[0].isStriped()); + checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); + + // save namespace, restart namenode, and check + dfs = cluster.getFileSystem(); + dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER); + dfs.saveNamespace(); + dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE); + cluster.restartNameNode(true); + fsdir = cluster.getNamesystem().getFSDirectory(); + fileNode = fsdir.getINode4Write(file.toString()).asFile(); + blocks = fileNode.getBlocks(); + assertEquals(1, blocks.length); + Assert.assertTrue(blocks[0].isStriped()); + checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); } finally { IOUtils.cleanup(null, out); } - - // restart NameNode to check editlog - cluster.restartNameNode(true); - FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); - INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); - BlockInfo[] blocks = fileNode.getBlocks(); - assertEquals(1, blocks.length); - Assert.assertTrue(blocks[0].isStriped()); - checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); - - // save namespace, restart namenode, and check - dfs = cluster.getFileSystem(); - dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER); - dfs.saveNamespace(); - dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE); - cluster.restartNameNode(true); - fsdir = cluster.getNamesystem().getFSDirectory(); - fileNode = fsdir.getINode4Write(file.toString()).asFile(); - blocks = fileNode.getBlocks(); - assertEquals(1, blocks.length); - Assert.assertTrue(blocks[0].isStriped()); - checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); } private void checkStripedBlockUC(BlockInfoStriped block, @@ -190,11 +190,12 @@ public void testGetLocatedStripedBlocks() throws Exception { FSDataOutputStream out = null; try { out = dfs.create(file, (short) 1); + DFSTestUtil.writeAndFlushStripedOutputStream( + (DFSStripedOutputStream) out.getWrappedStream(), + DFS_BYTES_PER_CHECKSUM_DEFAULT); FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); - cluster.getNamesystem().getAdditionalBlock(file.toString(), - fileNode.getId(), dfs.getClient().getClientName(), null, null, null); BlockInfoStripedUnderConstruction lastBlk = (BlockInfoStripedUnderConstruction) fileNode.getLastBlock(); DatanodeInfo[] expectedDNs = DatanodeStorageInfo From 77d94dd5ece2b12b0dfcc7e3b3094318115779b5 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 1 May 2015 15:59:58 -0700 Subject: [PATCH 093/212] HDFS-7949. WebImageViewer need support file size calculation with striped blocks. Contributed by Rakesh R. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../blockmanagement/BlockInfoStriped.java | 27 +-- .../offlineImageViewer/FSImageLoader.java | 21 ++- .../hadoop/hdfs/util/StripedBlockUtil.java | 22 +++ ...stOfflineImageViewerWithStripedBlocks.java | 166 ++++++++++++++++++ 5 files changed, 212 insertions(+), 27 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 596bbcfdd245d..145494f8108c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -155,3 +155,6 @@ HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() when loading editlog. (jing9) + + HDFS-7949. WebImageViewer need support file size calculation with striped + blocks. (Rakesh R via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 23e3153b57807..f0e52e370bf5e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -19,9 +19,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; - -import java.io.DataOutput; -import java.io.IOException; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @@ -203,28 +201,9 @@ public long spaceConsumed() { // In case striped blocks, total usage by this striped blocks should // be the total of data blocks and parity blocks because // `getNumBytes` is the total of actual data block size. - - // 0. Calculate the total bytes per stripes - long numBytesPerStripe = dataBlockNum * BLOCK_STRIPED_CELL_SIZE; - if (getNumBytes() % numBytesPerStripe == 0) { - return getNumBytes() / dataBlockNum * getTotalBlockNum(); + return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(), + dataBlockNum, parityBlockNum, BLOCK_STRIPED_CELL_SIZE); } - // 1. Calculate the number of stripes in this block group. - long numStripes = (getNumBytes() - 1) / numBytesPerStripe + 1; - // 2. Calculate the parity cell length in the last stripe. Note that the - // size of parity cells should equal the size of the first cell, if it - // is not full. - long lastStripeParityCellLen = Math.min(getNumBytes() % numBytesPerStripe, - BLOCK_STRIPED_CELL_SIZE); - // 3. Total consumed space is the total of - // - The total of the full cells of data blocks and parity blocks. - // - The remaining of data block which does not make a stripe. - // - The last parity block cells. These size should be same - // to the first cell in this stripe. - return getTotalBlockNum() * (BLOCK_STRIPED_CELL_SIZE * (numStripes - 1)) - + getNumBytes() % numBytesPerStripe - + lastStripeParityCellLen * parityBlockNum; - } @Override public final boolean isStriped() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java index 351ff032244db..42f6c0be27d3d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdfs.tools.offlineImageViewer; import java.io.BufferedInputStream; -import java.io.EOFException; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; @@ -42,12 +41,15 @@ import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf; import org.apache.hadoop.hdfs.server.namenode.FSImageUtil; import org.apache.hadoop.hdfs.server.namenode.FsImageProto; import org.apache.hadoop.hdfs.server.namenode.INodeId; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.hdfs.web.JsonUtil; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.LimitInputStream; @@ -483,8 +485,21 @@ private long lookup(String path) throws IOException { static long getFileSize(FsImageProto.INodeSection.INodeFile f) { long size = 0; - for (HdfsProtos.BlockProto p : f.getBlocksList()) { - size += p.getNumBytes(); + if (f.hasStripedBlocks()) { + List blocksList = f.getStripedBlocks().getBlocksList(); + // Get total of actual data block size + for (StripedBlockProto p : blocksList) { + // Total usage by this striped blocks should be the total of data + // blocks and parity blocks + size += StripedBlockUtil.spaceConsumedByStripedBlock(p.getBlock() + .getNumBytes(), p.getDataBlockNum(), p.getParityBlockNum(), + HdfsConstants.BLOCK_STRIPED_CELL_SIZE); + } + } else { + for (HdfsProtos.BlockProto p : f.getBlocksList()) { + size += p.getNumBytes(); + } + } return size; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index cb6d39a692452..b18e36f0fb4ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -231,6 +231,28 @@ public static StripedReadResult getNextCompletedStripedRead( } } + /** + * Get the total usage of the striped blocks, which is the total of data + * blocks and parity blocks + * + * @param numDataBlkBytes + * Size of the block group only counting data blocks + * @param dataBlkNum + * The number of data blocks + * @param parityBlkNum + * The number of parity blocks + * @param cellSize + * The size of a striping cell + * @return The total usage of data blocks and parity blocks + */ + public static long spaceConsumedByStripedBlock(long numDataBlkBytes, + int dataBlkNum, int parityBlkNum, int cellSize) { + int parityIndex = dataBlkNum + 1; + long numParityBlkBytes = getInternalBlockLength(numDataBlkBytes, cellSize, + dataBlkNum, parityIndex) * parityBlkNum; + return numDataBlkBytes + numParityBlkBytes; + } + /** * This class represents the portion of I/O associated with each block in the * striped block group. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java new file mode 100644 index 0000000000000..f3ef39a80c19d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java @@ -0,0 +1,166 @@ +/** + * 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.tools.offlineImageViewer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.UnresolvedLinkException; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; +import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.namenode.FSDirectory; +import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; +import org.apache.hadoop.hdfs.server.namenode.INodeFile; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestOfflineImageViewerWithStripedBlocks { + private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + + private static MiniDFSCluster cluster; + private static DistributedFileSystem fs; + private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final int stripesPerBlock = 3; + private static final int blockSize = cellSize * stripesPerBlock; + + @BeforeClass + public static void setup() throws IOException { + int numDNs = dataBlocks + parityBlocks + 2; + Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + fs = cluster.getFileSystem(); + Path eczone = new Path("/eczone"); + fs.mkdirs(eczone); + } + + @AfterClass + public static void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test(timeout = 60000) + public void testFileEqualToOneStripe() throws Exception { + int numBytes = cellSize; + testFileSize(numBytes); + } + + @Test(timeout = 60000) + public void testFileLessThanOneStripe() throws Exception { + int numBytes = cellSize - 100; + testFileSize(numBytes); + } + + @Test(timeout = 60000) + public void testFileHavingMultipleBlocks() throws Exception { + int numBytes = blockSize * 3; + testFileSize(numBytes); + } + + @Test(timeout = 60000) + public void testFileLargerThanABlockGroup1() throws IOException { + testFileSize(blockSize * dataBlocks + cellSize + 123); + } + + @Test(timeout = 60000) + public void testFileLargerThanABlockGroup2() throws IOException { + testFileSize(blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + + 123); + } + + @Test(timeout = 60000) + public void testFileFullBlockGroup() throws IOException { + testFileSize(blockSize * dataBlocks); + } + + @Test(timeout = 60000) + public void testFileMoreThanOneStripe() throws Exception { + int numBytes = blockSize + blockSize / 2; + testFileSize(numBytes); + } + + private void testFileSize(int numBytes) throws IOException, + UnresolvedLinkException, SnapshotAccessControlException { + fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE); + File orgFsimage = null; + Path file = new Path("/eczone/striped"); + FSDataOutputStream out = fs.create(file, true); + byte[] bytes = DFSTestUtil.generateSequentialBytes(0, numBytes); + out.write(bytes); + out.close(); + + // Write results to the fsimage file + fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false); + fs.saveNamespace(); + + // Determine location of fsimage file + orgFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil + .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0)); + if (orgFsimage == null) { + throw new RuntimeException("Didn't generate or can't find fsimage"); + } + FSImageLoader loader = FSImageLoader.load(orgFsimage.getAbsolutePath()); + String fileStatus = loader.getFileStatus("/eczone/striped"); + long expectedSpaceConsumed = StripedBlockUtil.spaceConsumedByStripedBlock( + bytes.length, HdfsConstants.NUM_DATA_BLOCKS, + HdfsConstants.NUM_PARITY_BLOCKS, HdfsConstants.BLOCK_STRIPED_CELL_SIZE); + + // Verify space consumed present in BlockInfoStriped + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + assertTrue("Invalid block size", fileNode.getBlocks().length > 0); + long actualSpaceConsumed = 0; + for (BlockInfo blockInfo : fileNode.getBlocks()) { + assertTrue("Didn't find block striped information", + blockInfo instanceof BlockInfoStriped); + BlockInfoStriped b = (BlockInfoStriped) blockInfo; + actualSpaceConsumed += b.spaceConsumed(); + } + + assertEquals("Wrongly computed file size contains striped blocks", + expectedSpaceConsumed, actualSpaceConsumed); + + // Verify space consumed present in filestatus + String EXPECTED_FILE_SIZE = "\"length\":" + + String.valueOf(expectedSpaceConsumed); + assertTrue( + "Wrongly computed file size contains striped blocks, file status:" + + fileStatus + ". Expected file size is : " + EXPECTED_FILE_SIZE, + fileStatus.contains(EXPECTED_FILE_SIZE)); + } +} From ea6c66ed5704dc8a9448b4c1b56e9664086c70fc Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 4 May 2015 11:24:35 -0700 Subject: [PATCH 094/212] HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249. Contributed by Zhe Zhang. --- .../apache/hadoop/hdfs/protocol/HdfsConstants.java | 11 +++++++++++ .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../apache/hadoop/hdfs/DFSStripedOutputStream.java | 2 +- .../hdfs/server/blockmanagement/BlockIdManager.java | 4 ++-- .../SequentialBlockGroupIdGenerator.java | 4 ++-- .../hdfs/server/common/HdfsServerConstants.java | 5 ++++- .../hdfs/server/namenode/TestAddStripedBlocks.java | 4 ++-- .../hdfs/server/namenode/TestStripedINodeFile.java | 6 +++--- 8 files changed, 28 insertions(+), 11 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java index 58c7ea1b89c98..32ca81c326749 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java @@ -75,6 +75,17 @@ public final class HdfsConstants { public static final String CLIENT_NAMENODE_PROTOCOL_NAME = "org.apache.hadoop.hdfs.protocol.ClientProtocol"; + /* + * These values correspond to the values used by the system default erasure + * coding schema. + * TODO: to be removed once all places use schema. + */ + + public static final byte NUM_DATA_BLOCKS = 6; + public static final byte NUM_PARITY_BLOCKS = 3; + // The chunk size for striped block which is used by erasure coding + public static final int BLOCK_STRIPED_CELL_SIZE = 256 * 1024; + // SafeMode actions public enum SafeModeAction { SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 145494f8108c6..e30b2ed2ec7ff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -158,3 +158,6 @@ HDFS-7949. WebImageViewer need support file size calculation with striped blocks. (Rakesh R via Zhe Zhang) + + HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249. + (Zhe Zhang via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 5e2a534ed78a2..71cdbb9d59add 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -419,7 +419,7 @@ void setClosed() { @Override protected synchronized void closeImpl() throws IOException { if (isClosed()) { - getLeadingStreamer().getLastException().check(); + getLeadingStreamer().getLastException().check(true); return; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index fc82d5da4c225..746802c74bba0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -241,11 +241,11 @@ public static boolean isStripedBlockID(long id) { * data/parity block id in the same striped block group. */ public static long convertToStripedID(long id) { - return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK); + return id & (~HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); } public static int getBlockIndex(Block reportedBlock) { return (int) (reportedBlock.getBlockId() & - HdfsConstants.BLOCK_GROUP_INDEX_MASK); + HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java index de8e379389be5..479ee4c923d21 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java @@ -21,8 +21,8 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.util.SequentialNumber; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_GROUP_INDEX_MASK; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_BLOCKS_IN_GROUP; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP; /** * Generate the next valid block group ID by incrementing the maximum block diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java index 2ebc92713aebc..40439280de1a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java @@ -396,6 +396,9 @@ enum BlockUCState { "raw.hdfs.crypto.file.encryption.info"; String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER = "security.hdfs.unreadable.by.superuser"; - public static final String XATTR_ERASURECODING_ZONE = + String XATTR_ERASURECODING_ZONE = "raw.hdfs.erasurecoding.zone"; + + long BLOCK_GROUP_INDEX_MASK = 15; + byte MAX_BLOCKS_IN_GROUP = 16; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 297db14978006..ba763604a83c0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -102,7 +102,7 @@ public void testAllocateBlockId() throws Exception { DFSTestUtil.writeFile(dfs, testPath, "hello again"); lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0); final long secondId = lb.get(0).getBlock().getBlockId(); - Assert.assertEquals(firstId + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId); + Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId); } @Test (timeout=60000) @@ -159,7 +159,7 @@ private void checkStripedBlockUC(BlockInfoStriped block, Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, block.getParityBlockNum()); Assert.assertEquals(0, - block.getBlockId() & HdfsConstants.BLOCK_GROUP_INDEX_MASK); + block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); final BlockInfoStripedUnderConstruction blockUC = (BlockInfoStripedUnderConstruction) block; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 4a6d6cc80d87a..f97943b58ed0b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -28,10 +28,10 @@ import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.Test; @@ -46,8 +46,8 @@ public class TestStripedINodeFile { "userName", null, FsPermission.getDefault()); private static INodeFile createStripedINodeFile() { - return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, - null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID); + return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, + null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID); } @Test From 6dea01f1ee5ea3bf6e146e1b68616c2f43ba4792 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 4 May 2015 14:44:58 -0700 Subject: [PATCH 095/212] HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSInputStream.java | 26 +++ .../hadoop/hdfs/DFSStripedInputStream.java | 217 ++++++++++++------ .../hadoop/hdfs/util/StripedBlockUtil.java | 34 ++- .../hdfs/TestDFSStripedInputStream.java | 50 +++- .../hadoop/hdfs/TestPlanReadPortions.java | 4 +- 6 files changed, 246 insertions(+), 88 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index e30b2ed2ec7ff..77272e7adb7b5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -161,3 +161,6 @@ HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249. (Zhe Zhang via jing9) + + HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout. + (jing9) 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 6e58cd6001909..0d51a5752da13 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 @@ -717,6 +717,16 @@ public synchronized int read() throws IOException { interface ReaderStrategy { public int doRead(BlockReader blockReader, int off, int len) throws ChecksumException, IOException; + + /** + * Copy data from the src ByteBuffer into the read buffer. + * @param src The src buffer where the data is copied from + * @param offset Useful only when the ReadStrategy is based on a byte array. + * Indicate the offset of the byte array for copy. + * @param length Useful only when the ReadStrategy is based on a byte array. + * Indicate the length of the data to copy. + */ + public int copyFrom(ByteBuffer src, int offset, int length); } protected void updateReadStatistics(ReadStatistics readStatistics, @@ -750,6 +760,13 @@ public int doRead(BlockReader blockReader, int off, int len) updateReadStatistics(readStatistics, nRead, blockReader); return nRead; } + + @Override + public int copyFrom(ByteBuffer src, int offset, int length) { + ByteBuffer writeSlice = src.duplicate(); + writeSlice.get(buf, offset, length); + return length; + } } /** @@ -783,6 +800,15 @@ public int doRead(BlockReader blockReader, int off, int len) } } } + + @Override + public int copyFrom(ByteBuffer src, int offset, int length) { + ByteBuffer writeSlice = src.duplicate(); + int remaining = Math.min(buf.remaining(), writeSlice.remaining()); + writeSlice.limit(writeSlice.position() + remaining); + buf.put(writeSlice); + return remaining; + } } /* This is a used by regular read() and handles ChecksumExceptions. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 0dc98fdbb00a5..13c4743040c9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs; +import com.google.common.base.Preconditions; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.*; @@ -37,6 +38,7 @@ import java.util.Map; import java.util.HashMap; import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutionException; import java.util.concurrent.CancellationException; @@ -62,7 +64,7 @@ * +------+ <- A cell contains {@link #cellSize} bytes of data * * Three styles of read will eventually be supported: - * 1. Stateful read: TODO: HDFS-8033 + * 1. Stateful read * 2. pread without decode support * This is implemented by calculating the portion of read from each block and * issuing requests to each DataNode in parallel. @@ -91,12 +93,38 @@ boolean shouldRefetchToken() { } } + /** Used to indicate the buffered data's range in the block group */ + private static class StripeRange { + /** start offset in the block group (inclusive) */ + final long offsetInBlock; + /** length of the stripe range */ + final long length; + + StripeRange(long offsetInBlock, long length) { + Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0); + this.offsetInBlock = offsetInBlock; + this.length = length; + } + + boolean include(long pos) { + return pos >= offsetInBlock && pos < offsetInBlock + length; + } + } + private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS; private final BlockReader[] blockReaders = new BlockReader[groupSize]; private final DatanodeInfo[] currentNodes = new DatanodeInfo[groupSize]; private final int cellSize; private final short dataBlkNum; private final short parityBlkNum; + /** the buffer for a complete stripe */ + private ByteBuffer curStripeBuf; + /** + * indicate the start/end offset of the current buffered stripe in the + * block group + */ + private StripeRange curStripeRange; + private final CompletionService readingService; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECInfo ecInfo) throws IOException { @@ -106,7 +134,20 @@ boolean shouldRefetchToken() { cellSize = ecInfo.getSchema().getChunkSize(); dataBlkNum = (short) ecInfo.getSchema().getNumDataUnits(); parityBlkNum = (short) ecInfo.getSchema().getNumParityUnits(); - DFSClient.LOG.debug("Creating an striped input stream for file " + src); + curStripeRange = new StripeRange(0, 0); + readingService = + new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Creating an striped input stream for file " + src); + } + } + + private void resetCurStripeBuffer() { + if (curStripeBuf == null) { + curStripeBuf = ByteBuffer.allocateDirect(cellSize * dataBlkNum); + } + curStripeBuf.clear(); + curStripeRange = new StripeRange(0, 0); } @Override @@ -141,7 +182,7 @@ private synchronized void blockSeekTo(long target) throws IOException { targetBlockGroup.getBlockSize() - 1; currentLocatedBlock = targetBlockGroup; - long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset(); + final long offsetIntoBlockGroup = getOffsetInBlockGroup(); LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup( targetBlockGroup, cellSize, dataBlkNum, parityBlkNum); // The purpose is to get start offset into each block @@ -156,8 +197,8 @@ private synchronized void blockSeekTo(long target) throws IOException { if (retval != null) { currentNodes[i] = retval.info; blockReaders[i] = getBlockReaderWithRetry(targetBlock, - readPortions[i].startOffsetInBlock, - targetBlock.getBlockSize() - readPortions[i].startOffsetInBlock, + readPortions[i].getStartOffsetInBlock(), + targetBlock.getBlockSize() - readPortions[i].getStartOffsetInBlock(), retval.addr, retval.storageType, retval.info, target, retry); } } @@ -203,6 +244,7 @@ private BlockReader getBlockReaderWithRetry(LocatedBlock targetBlock, */ @Override protected void closeCurrentBlockReaders() { + resetCurStripeBuffer(); if (blockReaders == null || blockReaders.length == 0) { return; } @@ -220,6 +262,73 @@ protected void closeCurrentBlockReaders() { blockEnd = -1; } + private long getOffsetInBlockGroup() { + return pos - currentLocatedBlock.getStartOffset(); + } + + /** + * Read a new stripe covering the current position, and store the data in the + * {@link #curStripeBuf}. + */ + private void readOneStripe( + Map> corruptedBlockMap) + throws IOException { + resetCurStripeBuffer(); + + // compute stripe range based on pos + final long offsetInBlockGroup = getOffsetInBlockGroup(); + final long stripeLen = cellSize * dataBlkNum; + int stripeIndex = (int) (offsetInBlockGroup / stripeLen); + curStripeRange = new StripeRange(stripeIndex * stripeLen, + Math.min(currentLocatedBlock.getBlockSize() - (stripeIndex * stripeLen), + stripeLen)); + final int numCell = (int) ((curStripeRange.length - 1) / cellSize + 1); + + // read the whole stripe in parallel + Map, Integer> futures = new HashMap<>(); + for (int i = 0; i < numCell; i++) { + curStripeBuf.position(cellSize * i); + curStripeBuf.limit((int) Math.min(cellSize * (i + 1), + curStripeRange.length)); + ByteBuffer buf = curStripeBuf.slice(); + ByteBufferStrategy strategy = new ByteBufferStrategy(buf); + final int targetLength = buf.remaining(); + Callable readCallable = readCell(blockReaders[i], + currentNodes[i], strategy, targetLength, corruptedBlockMap); + Future request = readingService.submit(readCallable); + futures.put(request, i); + } + while (!futures.isEmpty()) { + try { + waitNextCompletion(readingService, futures); + // TODO: decode and record bad reader if necessary + } catch (InterruptedException ignored) { + // ignore and retry + } + } + } + + private Callable readCell(final BlockReader reader, + final DatanodeInfo datanode, final ByteBufferStrategy strategy, + final int targetLength, + final Map> corruptedBlockMap) { + return new Callable() { + @Override + public Integer call() throws Exception { + int result = 0; + while (result < targetLength) { + int ret = readBuffer(reader, datanode, strategy, corruptedBlockMap); + if (ret < 0) { + throw new IOException("Unexpected EOS from the reader"); + } + result += ret; + } + updateReadStatistics(readStatistics, targetLength, reader); + return result; + } + }; + } + @Override protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException { @@ -227,11 +336,10 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, if (closed.get()) { throw new IOException("Stream closed"); } - Map> corruptedBlockMap = new HashMap<>(); + Map> corruptedBlockMap = + new ConcurrentHashMap<>(); failures = 0; if (pos < getFileLength()) { - /** Index of the target block in a stripe to read from */ - int idxInGroup = (int) ((pos / cellSize) % dataBlkNum); try { if (pos > blockEnd) { blockSeekTo(pos); @@ -247,40 +355,13 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, /** Number of bytes already read into buffer */ int result = 0; while (result < realLen) { - /** - * Temporary position into the file; {@link pos} might not proceed - * to this temporary position in case of exceptions. - */ - long tmpPos = pos + result; - /** Start and end offsets of a cell in the file */ - long cellStart = (tmpPos / cellSize) * cellSize; - long cellEnd = cellStart + cellSize - 1; - - /** Number of bytes to read from the current cell */ - int realLenInCell = (int) Math.min(realLen - result, - cellEnd - tmpPos + 1L); - assert realLenInCell > 0 : "Temporary position shouldn't be " - + "after cellEnd"; - - // Read from one blockReader up to cell boundary - int cellRet = readBuffer(blockReaders[idxInGroup], - currentNodes[idxInGroup], strategy, off + result, realLenInCell, - corruptedBlockMap); - if (cellRet >= 0) { - result += cellRet; - if (cellRet < realLenInCell) { - // A short read indicates the current blockReader buffer is - // already drained. Should return the read call. Otherwise - // should proceed to the next cell. - break; - } - } else { - // got a EOS from reader though we expect more data on it. - throw new IOException("Unexpected EOS from the reader"); + if (!curStripeRange.include(getOffsetInBlockGroup())) { + readOneStripe(corruptedBlockMap); } - idxInGroup = (idxInGroup + 1) % dataBlkNum; + int ret = copy(strategy, off + result, realLen - result); + result += ret; + pos += ret; } - pos += result; if (dfsClient.stats != null) { dfsClient.stats.incrementBytesRead(result); } @@ -295,11 +376,11 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, return -1; } - private synchronized int readBuffer(BlockReader blockReader, - DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len, + private int readBuffer(BlockReader blockReader, + DatanodeInfo currentNode, ByteBufferStrategy readerStrategy, Map> corruptedBlockMap) { try { - return readerStrategy.doRead(blockReader, off, len); + return readerStrategy.doRead(blockReader, 0, 0); } catch ( ChecksumException ce ) { DFSClient.LOG.warn("Found Checksum error for " + getCurrentBlock() + " from " + currentNode @@ -312,26 +393,25 @@ private synchronized int readBuffer(BlockReader blockReader, + getCurrentBlock() + " of " + src + " from " + currentNode, e); } - // TODO: this should trigger decoding logic (HDFS-7678) return -1; } - protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy { - ByteBufferStrategy(ByteBuffer buf) { - super(buf); - } - - @Override - public int doRead(BlockReader blockReader, int off, int len) - throws IOException { - int oldlimit = buf.limit(); - if (buf.remaining() > len) { - buf.limit(buf.position() + len); - } - int ret = super.doRead(blockReader, off, len); - buf.limit(oldlimit); - return ret; - } + /** + * Copy the data from {@link #curStripeBuf} into the given buffer + * @param strategy the ReaderStrategy containing the given buffer + * @param offset the offset of the given buffer. Used only when strategy is + * a ByteArrayStrategy + * @param length target length + * @return number of bytes copied + */ + private int copy(ReaderStrategy strategy, int offset, int length) { + final long stripeLen = cellSize * dataBlkNum; + final long offsetInBlk = pos - currentLocatedBlock.getStartOffset(); + // compute the position in the curStripeBuf based on "pos" + int bufOffset = (int) (offsetInBlk % stripeLen); + curStripeBuf.position(bufOffset); + return strategy.copyFrom(curStripeBuf, offset, + Math.min(length, curStripeBuf.remaining())); } /** @@ -366,8 +446,7 @@ protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { DFSClient.LOG.debug("getBlockAt for striped blocks, offset=" + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx); } - return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize, - dataBlkNum, idx); + return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize, dataBlkNum, idx); } private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException { @@ -404,7 +483,7 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, for (short i = 0; i < dataBlkNum; i++) { ReadPortion rp = readPortions[i]; - if (rp.readLength <= 0) { + if (rp.getReadLength() <= 0) { continue; } DatanodeInfo loc = blks[i].getLocations()[0]; @@ -413,8 +492,8 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), type); Callable readCallable = getFromOneDataNode(dnAddr, - blks[i].getStartOffset(), rp.startOffsetInBlock, - rp.startOffsetInBlock + rp.readLength - 1, buf, + blks[i].getStartOffset(), rp.getStartOffsetInBlock(), + rp.getStartOffsetInBlock() + rp.getReadLength() - 1, buf, rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i); Future getFromDNRequest = stripedReadsService.submit(readCallable); DFSClient.LOG.debug("Submitting striped read request for " + blks[i]); @@ -451,14 +530,14 @@ public Void call() throws Exception { }; } - private void waitNextCompletion(CompletionService stripedReadsService, - Map, Integer> futures) throws InterruptedException { + private void waitNextCompletion(CompletionService service, + Map, Integer> futures) throws InterruptedException { if (futures.isEmpty()) { throw new InterruptedException("Futures already empty"); } - Future future = null; + Future future = null; try { - future = stripedReadsService.take(); + future = service.take(); future.get(); futures.remove(future); } catch (ExecutionException | CancellationException e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index b18e36f0fb4ac..24d4bfba844ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -169,22 +169,22 @@ public static ReadPortion[] planReadPortions(final int dataBlkNum, // blkIdxInGroup is the index of the block in the striped block group // E.g., blk_2 is the 3rd block in the group final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum); - results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk + - startInBlk % cellSize; + results[blkIdxInGroup].setStartOffsetInBlock(cellSize * cellIdxInBlk + + startInBlk % cellSize); boolean crossStripe = false; for (int i = 1; i < dataBlkNum; i++) { if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) { cellIdxInBlk++; crossStripe = true; } - results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock = - cellSize * cellIdxInBlk; + results[(blkIdxInGroup + i) % dataBlkNum].setStartOffsetInBlock( + cellSize * cellIdxInBlk); } int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len); results[blkIdxInGroup].offsetsInBuf.add(bufOffset); results[blkIdxInGroup].lengths.add(firstCellLen); - results[blkIdxInGroup].readLength += firstCellLen; + results[blkIdxInGroup].addReadLength(firstCellLen); int i = (blkIdxInGroup + 1) % dataBlkNum; for (int done = firstCellLen; done < len; done += cellSize) { @@ -192,7 +192,7 @@ public static ReadPortion[] planReadPortions(final int dataBlkNum, rp.offsetsInBuf.add(done + bufOffset); final int readLen = Math.min(len - done, cellSize); rp.lengths.add(readLen); - rp.readLength += readLen; + rp.addReadLength(readLen); i = (i + 1) % dataBlkNum; } return results; @@ -274,8 +274,8 @@ public static class ReadPortion { * | (partial) | (from blk_1 and blk_2) | | * +------------------------------------------------------+ */ - public long startOffsetInBlock = 0; - public int readLength = 0; + private long startOffsetInBlock = 0; + private int readLength = 0; public final List offsetsInBuf = new ArrayList<>(); public final List lengths = new ArrayList<>(); @@ -295,10 +295,20 @@ public int[] getLengths() { return lens; } - public boolean containsReadPortion(ReadPortion rp) { - long end = startOffsetInBlock + readLength; - return startOffsetInBlock <= rp.startOffsetInBlock && end >= - rp.startOffsetInBlock + rp.readLength; + public long getStartOffsetInBlock() { + return startOffsetInBlock; + } + + public int getReadLength() { + return readLength; + } + + public void setStartOffsetInBlock(long startOffsetInBlock) { + this.startOffsetInBlock = startOffsetInBlock; + } + + void addReadLength(int extraLength) { + this.readLength += extraLength; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index bcfc74b3d5bbf..11cdf7b498563 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -158,7 +158,7 @@ private byte getByte(long pos) { private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) throws IOException { Path testPath = new Path(src); - byte[] bytes = generateBytes(writeBytes); + final byte[] bytes = generateBytes(writeBytes); DFSTestUtil.writeFile(fs, testPath, new String(bytes)); //check file length @@ -175,7 +175,8 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) Assert.assertEquals("The length of file should be the same to write size", writeBytes, readLen); for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]); + Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), + buf[i]); } } @@ -190,12 +191,12 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) readLen += ret; } } while (ret >= 0); - readLen = readLen >= 0 ? readLen : 0; Assert.assertEquals("The length of file should be the same to write size", writeBytes, readLen); for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]); + Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), + buf[i]); } } @@ -214,8 +215,47 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) Assert.assertEquals("The length of file should be the same to write size", writeBytes, readLen); for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at i should be the same", getByte(i), buf.array()[i]); + Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), + buf.array()[i]); } } + + // stateful read with 1KB size byte array + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + final byte[] result = new byte[writeBytes]; + final byte[] buf = new byte[1024]; + int readLen = 0; + int ret; + do { + ret = fsdis.read(buf, 0, buf.length); + if (ret > 0) { + System.arraycopy(buf, 0, result, readLen, ret); + readLen += ret; + } + } while (ret >= 0); + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + Assert.assertArrayEquals(bytes, result); + } + + // stateful read using ByteBuffer with 1KB size + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + final ByteBuffer result = ByteBuffer.allocate(writeBytes); + final ByteBuffer buf = ByteBuffer.allocate(1024); + int readLen = 0; + int ret; + do { + ret = fsdis.read(buf); + if (ret > 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } while (ret >= 0); + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + Assert.assertArrayEquals(bytes, result.array()); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java index 3b5787a4a94f1..75d05879f213e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java @@ -38,8 +38,8 @@ private void testPlanReadPortions(int startInBlk, int length, assertEquals(GROUP_SIZE, results.length); for (int i = 0; i < GROUP_SIZE; i++) { - assertEquals(readLengths[i], results[i].readLength); - assertEquals(offsetsInBlock[i], results[i].startOffsetInBlock); + assertEquals(readLengths[i], results[i].getReadLength()); + assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock()); final int[] bOffsets = results[i].getOffsets(); assertArrayEquals(bufferOffsets[i], bOffsets); final int[] bLengths = results[i].getLengths(); From 436c14855aef58a551d5cab91eba6e88775c6797 Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Tue, 5 May 2015 11:22:52 +0530 Subject: [PATCH 096/212] HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command. Contributed by Uma Maheswara Rao G --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/hdfs/protocolPB/PBHelper.java | 6 ++- .../server/blockmanagement/BlockManager.java | 22 +++++++++- .../blockmanagement/DatanodeDescriptor.java | 16 +++---- .../hdfs/server/namenode/FSNamesystem.java | 43 +++++++++++-------- .../hdfs/server/namenode/Namesystem.java | 14 +++++- .../protocol/BlockECRecoveryCommand.java | 14 +++++- .../src/main/proto/erasurecoding.proto | 1 + .../hadoop/hdfs/protocolPB/TestPBHelper.java | 21 +++++++-- 9 files changed, 102 insertions(+), 37 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 77272e7adb7b5..faec023add4bb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -164,3 +164,5 @@ HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout. (jing9) + + HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command(umamahesh) 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 3cd3e03d1bcad..e230232767fc5 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 @@ -3191,8 +3191,10 @@ public static BlockECRecoveryInfo convertBlockECRecoveryInfo( liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue(); } + ECSchema ecSchema = convertECSchema(blockEcRecoveryInfoProto.getEcSchema()); + return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos, - targetStorageUuids, convertStorageTypes, liveBlkIndices); + targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema); } public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( @@ -3217,6 +3219,8 @@ public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices(); builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices)); + builder.setEcSchema(convertECSchema(blockEcRecoveryInfo.getECSchema())); + return builder.build(); } 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 6657e5b369491..a3d75b5ae74f1 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 @@ -66,7 +66,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -84,7 +83,10 @@ 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.io.erasurecode.ECSchema; + import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; + import org.apache.hadoop.net.Node; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Daemon; @@ -94,6 +96,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Sets; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1554,10 +1557,25 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { if (block.isStriped()) { assert rw instanceof ErasureCodingWork; assert rw.targets.length > 0; + String src = block.getBlockCollection().getName(); + ECSchema ecSchema = null; + try { + ecSchema = namesystem.getECSchemaForPath(src); + } catch (IOException e) { + blockLog + .warn("Failed to get the EC schema for the file {} ", src); + } + if (ecSchema == null) { + blockLog.warn("No EC schema found for the file {}. " + + "So cannot proceed for recovery", src); + // TODO: we may have to revisit later for what we can do better to + // handle this case. + continue; + } rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded( new ExtendedBlock(namesystem.getBlockPoolId(), block), rw.srcNodes, rw.targets, - ((ErasureCodingWork) rw).liveBlockIndicies); + ((ErasureCodingWork) rw).liveBlockIndicies, ecSchema); } else { rw.srcNodes[0].addBlockToBeReplicated(block, targets); } 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 35cc31b7088ec..83d33035bdd6f 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 @@ -29,7 +29,6 @@ import java.util.Map; import java.util.Queue; import java.util.Set; -import java.util.Arrays; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; @@ -51,6 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; import org.apache.hadoop.hdfs.util.EnumCounters; import org.apache.hadoop.hdfs.util.LightWeightHashSet; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.IntrusiveCollection; import org.apache.hadoop.util.Time; @@ -608,15 +608,15 @@ void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { /** * Store block erasure coding work. */ - void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources, - DatanodeStorageInfo[] targets, short[] liveBlockIndices) { - assert(block != null && sources != null && sources.length > 0); + void addBlockToBeErasureCoded(ExtendedBlock block, + DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets, + short[] liveBlockIndices, ECSchema ecSchema) { + assert (block != null && sources != null && sources.length > 0); BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets, - liveBlockIndices); + liveBlockIndices, ecSchema); erasurecodeBlocks.offer(task); - BlockManager.LOG.debug("Adding block recovery task " + task + - "to " + getName() + ", current queue size is " + - erasurecodeBlocks.size()); + BlockManager.LOG.debug("Adding block recovery task " + task + "to " + + getName() + ", current queue size is " + erasurecodeBlocks.size()); } /** 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 3fa88184a923c..79dd3d7803044 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 @@ -7581,25 +7581,9 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, */ ECInfo getErasureCodingInfo(String src) throws AccessControlException, UnresolvedLinkException, IOException { - checkOperation(OperationCategory.READ); - final byte[][] pathComponents = FSDirectory - .getPathComponentsForReservedPath(src); - final FSPermissionChecker pc = getPermissionChecker(); - readLock(); - try { - checkOperation(OperationCategory.READ); - src = dir.resolvePath(pc, src, pathComponents); - final INodesInPath iip = dir.getINodesInPath(src, true); - if (isPermissionEnabled) { - dir.checkPathAccess(pc, iip, FsAction.READ); - } - // Get schema set for the zone - ECSchema schema = dir.getECSchema(iip); - if (schema != null) { - return new ECInfo(src, schema); - } - } finally { - readUnlock(); + ECSchema schema = getECSchemaForPath(src); + if (schema != null) { + return new ECInfo(src, schema); } return null; } @@ -7841,5 +7825,26 @@ private static void enableAsyncAuditLog() { } } + @Override + public ECSchema getECSchemaForPath(String src) throws IOException { + checkOperation(OperationCategory.READ); + final byte[][] pathComponents = FSDirectory + .getPathComponentsForReservedPath(src); + final FSPermissionChecker pc = getPermissionChecker(); + readLock(); + try { + checkOperation(OperationCategory.READ); + src = dir.resolvePath(pc, src, pathComponents); + final INodesInPath iip = dir.getINodesInPath(src, true); + if (isPermissionEnabled) { + dir.checkPathAccess(pc, iip, FsAction.READ); + } + // Get schema set for the zone + return dir.getECSchema(iip); + } finally { + readUnlock(); + } + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java index 4695c3f7dbf19..e6c7fc0f17a34 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import java.io.IOException; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.util.RwLock; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.StandbyException; import org.apache.hadoop.security.AccessControlException; @@ -47,4 +49,14 @@ public interface Namesystem extends RwLock, SafeMode { public void checkOperation(OperationCategory read) throws StandbyException; public boolean isInSnapshot(BlockCollection bc); + + /** + * Gets the ECSchema for the specified path + * + * @param src + * - path + * @return ECSchema + * @throws IOException + */ + public ECSchema getECSchemaForPath(String src) throws IOException; } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java index 9a387dd9e0d85..61e49e933de66 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.io.erasurecode.ECSchema; import java.util.Arrays; import java.util.Collection; @@ -76,9 +77,11 @@ public static class BlockECRecoveryInfo { private String[] targetStorageIDs; private StorageType[] targetStorageTypes; private final short[] liveBlockIndices; + private final ECSchema ecSchema; public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, - DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices) { + DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices, + ECSchema ecSchema) { this.block = block; this.sources = sources; this.targets = DatanodeStorageInfo.toDatanodeInfos(targetDnStorageInfo); @@ -87,17 +90,20 @@ public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, this.targetStorageTypes = DatanodeStorageInfo .toStorageTypes(targetDnStorageInfo); this.liveBlockIndices = liveBlockIndices; + this.ecSchema = ecSchema; } public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, DatanodeInfo[] targets, String[] targetStorageIDs, - StorageType[] targetStorageTypes, short[] liveBlockIndices) { + StorageType[] targetStorageTypes, short[] liveBlockIndices, + ECSchema ecSchema) { this.block = block; this.sources = sources; this.targets = targets; this.targetStorageIDs = targetStorageIDs; this.targetStorageTypes = targetStorageTypes; this.liveBlockIndices = liveBlockIndices; + this.ecSchema = ecSchema; } public ExtendedBlock getExtendedBlock() { @@ -123,6 +129,10 @@ public StorageType[] getTargetStorageTypes() { public short[] getLiveBlockIndices() { return liveBlockIndices; } + + public ECSchema getECSchema() { + return ecSchema; + } @Override public String toString() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 59bd9497ef0cd..702f6fd44c788 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -99,4 +99,5 @@ message BlockECRecoveryInfoProto { required StorageUuidsProto targetStorageUuids = 4; required StorageTypesProto targetStorageTypes = 5; repeated uint32 liveBlockIndices = 6; + required ECSchemaProto ecSchema = 7; } \ No newline at end of file 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 4ec4ea5214f0b..f580cbbebd01e 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 @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -71,8 +70,8 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.StorageInfo; -import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; +import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; @@ -88,6 +87,7 @@ 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.io.erasurecode.ECSchema; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; @@ -663,7 +663,7 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices0 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo( new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0, - liveBlkIndices0); + liveBlkIndices0, ECSchemaManager.getSystemDefaultSchema()); DatanodeInfo[] dnInfos1 = new DatanodeInfo[] { DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() }; DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil @@ -677,7 +677,7 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices1 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo( new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1, - liveBlkIndices1); + liveBlkIndices1, ECSchemaManager.getSystemDefaultSchema()); List blkRecoveryInfosList = new ArrayList(); blkRecoveryInfosList.add(blkECRecoveryInfo0); blkRecoveryInfosList.add(blkECRecoveryInfo1); @@ -718,6 +718,19 @@ private void assertBlockECRecoveryInfoEquals( for (int i = 0; i < liveBlockIndices1.length; i++) { assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]); } + + ECSchema ecSchema1 = blkECRecoveryInfo1.getECSchema(); + ECSchema ecSchema2 = blkECRecoveryInfo2.getECSchema(); + // Compare ECSchemas same as default ECSchema as we used system default + // ECSchema used in this test + compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema1); + compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema2); + } + + private void compareECSchemas(ECSchema ecSchema1, ECSchema ecSchema2) { + assertEquals(ecSchema1.getSchemaName(), ecSchema2.getSchemaName()); + assertEquals(ecSchema1.getNumDataUnits(), ecSchema2.getNumDataUnits()); + assertEquals(ecSchema1.getNumParityUnits(), ecSchema2.getNumParityUnits()); } private void assertDnInfosEqual(DatanodeInfo[] dnInfos1, From 4392325546a49d29720fe705381447a0fdd5826e Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 5 May 2015 11:54:30 +0530 Subject: [PATCH 097/212] HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands (Contributed by Rakesh R) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hdfs/tools/erasurecode/ECCommand.java | 9 +- .../hadoop/cli/CLITestCmdErasureCoding.java | 38 ++ .../hadoop/cli/TestErasureCodingCLI.java | 114 ++++++ .../cli/util/CLICommandErasureCodingCli.java | 21 ++ .../cli/util/ErasureCodingCliCmdExecutor.java | 37 ++ .../test/resources/testErasureCodingConf.xml | 342 ++++++++++++++++++ 7 files changed, 561 insertions(+), 3 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index faec023add4bb..ef760fcd9c241 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -166,3 +166,6 @@ (jing9) HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command(umamahesh) + + HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands + (Rakesh R via vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index 84c22751a38c1..802a46dd08c13 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -17,7 +17,9 @@ package org.apache.hadoop.hdfs.tools.erasurecode; import java.io.IOException; +import java.util.ArrayList; import java.util.LinkedList; +import java.util.List; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; @@ -120,11 +122,12 @@ protected void processPath(PathData item) throws IOException { sb.append("Schema '"); sb.append(schemaName); sb.append("' does not match any of the supported schemas."); - sb.append("Please select any one of ["); + sb.append(" Please select any one of "); + List schemaNames = new ArrayList(); for (ECSchema ecSchema : ecSchemas) { - sb.append(ecSchema.getSchemaName()); - sb.append(", "); + schemaNames.add(ecSchema.getSchemaName()); } + sb.append(schemaNames); throw new HadoopIllegalArgumentException(sb.toString()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java new file mode 100644 index 0000000000000..6c06a8da85aa7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java @@ -0,0 +1,38 @@ +/** + * 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.cli; + +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli; +import org.apache.hadoop.cli.util.CLICommandTypes; +import org.apache.hadoop.cli.util.CLITestCmd; +import org.apache.hadoop.cli.util.CommandExecutor; +import org.apache.hadoop.cli.util.ErasureCodingCliCmdExecutor; +import org.apache.hadoop.hdfs.tools.erasurecode.ECCli; + +public class CLITestCmdErasureCoding extends CLITestCmd { + public CLITestCmdErasureCoding(String str, CLICommandTypes type) { + super(str, type); + } + + @Override + public CommandExecutor getExecutor(String tag) throws IllegalArgumentException { + if (getType() instanceof CLICommandErasureCodingCli) + return new ErasureCodingCliCmdExecutor(tag, new ECCli()); + return super.getExecutor(tag); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java new file mode 100644 index 0000000000000..5f01ea291641d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java @@ -0,0 +1,114 @@ +/** + * 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.cli; + +import org.apache.hadoop.cli.util.CLICommand; +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli; +import org.apache.hadoop.cli.util.CommandExecutor.Result; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.xml.sax.SAXException; + +public class TestErasureCodingCLI extends CLITestHelper { + private final int NUM_OF_DATANODES = 3; + private MiniDFSCluster dfsCluster = null; + private FileSystem fs = null; + private String namenode = null; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + + dfsCluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(NUM_OF_DATANODES).build(); + dfsCluster.waitClusterUp(); + namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///"); + + username = System.getProperty("user.name"); + + fs = dfsCluster.getFileSystem(); + } + + @Override + protected String getTestFile() { + return "testErasureCodingConf.xml"; + } + + @After + @Override + public void tearDown() throws Exception { + if (fs != null) { + fs.close(); + } + if (dfsCluster != null) { + dfsCluster.shutdown(); + } + Thread.sleep(2000); + super.tearDown(); + } + + @Override + protected String expandCommand(final String cmd) { + String expCmd = cmd; + expCmd = expCmd.replaceAll("NAMENODE", namenode); + expCmd = expCmd.replaceAll("#LF#", System.getProperty("line.separator")); + expCmd = super.expandCommand(expCmd); + return expCmd; + } + + @Override + protected TestConfigFileParser getConfigParser() { + return new TestErasureCodingAdmin(); + } + + private class TestErasureCodingAdmin extends + CLITestHelper.TestConfigFileParser { + @Override + public void endElement(String uri, String localName, String qName) + throws SAXException { + if (qName.equals("ec-admin-command")) { + if (testCommands != null) { + testCommands.add(new CLITestCmdErasureCoding(charString, + new CLICommandErasureCodingCli())); + } else if (cleanupCommands != null) { + cleanupCommands.add(new CLITestCmdErasureCoding(charString, + new CLICommandErasureCodingCli())); + } + } else { + super.endElement(uri, localName, qName); + } + } + } + + @Override + protected Result execute(CLICommand cmd) throws Exception { + return cmd.getExecutor(namenode).executeCommand(cmd.getCmd()); + } + + @Test + @Override + public void testAll() { + super.testAll(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java new file mode 100644 index 0000000000000..aafcd9fa77458 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java @@ -0,0 +1,21 @@ +/** + * 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.cli.util; + +public class CLICommandErasureCodingCli implements CLICommandTypes { +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java new file mode 100644 index 0000000000000..e993313a8fbc6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java @@ -0,0 +1,37 @@ +/** + * 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.cli.util; + +import org.apache.hadoop.hdfs.tools.erasurecode.ECCli; +import org.apache.hadoop.util.ToolRunner; + +public class ErasureCodingCliCmdExecutor extends CommandExecutor { + protected String namenode = null; + protected ECCli admin = null; + + public ErasureCodingCliCmdExecutor(String namenode, ECCli admin) { + this.namenode = namenode; + this.admin = admin; + } + + @Override + protected void execute(final String cmd) throws Exception { + String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode); + ToolRunner.run(admin, args); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml new file mode 100644 index 0000000000000..b7b29d3a6961c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -0,0 +1,342 @@ + + + + + + + + test + + + + + + + help: help for erasure coding command + + -help + + + + + + SubstringComparator + Usage: hdfs erasurecode [generic options] + + + + + + help: createZone command + + -fs NAMENODE -help createZone + + + + + + RegexpComparator + ^[ \t]*Create a zone to encode files using a specified schema( )* + + + RegexpComparator + ^-createZone \[-s <schemaName>\] <path>(.)* + + + + + + help: getZoneInfo command + + -fs NAMENODE -help getZoneInfo + + + + + + SubstringComparator + Get information about the EC zone at specified path + + + RegexpComparator + ^-getZoneInfo <path>(.)* + + + + + + help: listSchemas command + + -fs NAMENODE -help listSchemas + + + + + + SubstringComparator + Get the list of ECSchemas supported + + + RegexpComparator + ^-listSchemas (.)* + + + + + + + createZone : create a zone to encode files + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone -s RS-6-3 /eczone + + + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + EC Zone created successfully at NAMENODE/eczone + + + + + + createZone : default schema + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone /eczone + -fs NAMENODE -getZoneInfo /eczone + + + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + Dir: /eczone, Schema: ECSchema=[Name=RS-6-3 + + + + + + getZoneInfo : get information about the EC zone at specified path + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone -s RS-6-3 /eczone + -fs NAMENODE -getZoneInfo /eczone + + + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + Dir: /eczone, Schema: ECSchema=[Name=RS-6-3 + + + + + + getZoneInfo : get EC zone at specified file path + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone -s RS-6-3 /eczone + -fs NAMENODE -touchz /eczone/ecfile + -fs NAMENODE -getZoneInfo /eczone/ecfile + + + -fs NAMENODE -rm /eczone/ecfile + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + Dir: /eczone, Schema: ECSchema=[Name=RS-6-3 + + + + + + listSchemas : get the list of ECSchemas supported + + -fs NAMENODE -listSchemas + + + + + + SubstringComparator + RS-6-3 + + + + + + + createZone : illegal parameters - path is missing + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone + + + -fs NAMENODE -rmdir /eczone + + + + RegexpComparator + ^-createZone: <path> is missing(.)* + + + + + + createZone : illegal parameters - schema name is missing + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone -s + + + -fs NAMENODE -rmdir /eczone + + + + RegexpComparator + ^-createZone: option -s requires 1 argument(.)* + + + + + + createZone : illegal parameters - too many arguments + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone /eczone1 /eczone2 + + + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + -createZone: Too many arguments + + + + + + createZone : illegal parameters - invalidschema + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone -s invalidschema /eczone + + + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + Schema 'invalidschema' does not match any of the supported schemas. Please select any one of [RS-6-3] + + + + + + createZone : illegal parameters - no such file + + -fs NAMENODE -createZone /eczone + + + + + + RegexpComparator + ^createZone: `/eczone': No such file or directory(.)* + + + + + + getZoneInfo : illegal parameters - path is missing + + -fs NAMENODE -getZoneInfo + + + + + + RegexpComparator + ^-getZoneInfo: <path> is missing(.)* + + + + + + getZoneInfo : illegal parameters - too many arguments + + -fs NAMENODE -getZoneInfo /eczone /eczone + + + -fs NAMENODE -rm /eczone + + + + SubstringComparator + -getZoneInfo: Too many arguments + + + + + + getZoneInfo : illegal parameters - no such file + + -fs NAMENODE -getZoneInfo /eczone + + + + + + RegexpComparator + ^getZoneInfo: `/eczone': No such file or directory(.)* + + + + + + listSchemas : illegal parameters - too many parameters + + -fs NAMENODE -listSchemas /eczone + + + + + + SubstringComparator + -listSchemas: Too many parameters + + + + + + From e849be2d31b6b5cda14f3c19b5469ca4d2742d7d Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Tue, 5 May 2015 19:25:21 +0530 Subject: [PATCH 098/212] HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..). Contributed by Vinayakumar B --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index ef760fcd9c241..a8df3f268f9a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -169,3 +169,6 @@ HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands (Rakesh R via vinayakumarb) + + HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via + umamahesh) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 722bb39695d13..84e7871e53731 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -3358,11 +3358,14 @@ TraceScope getSrcDstTraceScope(String description, String src, String dst) { */ public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { checkOpen(); + TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src); try { return namenode.getErasureCodingZoneInfo(src); } catch (RemoteException re) { throw re.unwrapRemoteException(FileNotFoundException.class, AccessControlException.class, UnresolvedPathException.class); + } finally { + scope.close(); } } } From 220ca960bce970d5969b9af570a3ce43360b7e2b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 5 May 2015 16:26:49 -0700 Subject: [PATCH 099/212] HDFS-7672. Handle write failure for stripping blocks and refactor the existing code in DFSStripedOutputStream and StripedDataStreamer. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSOutputStream.java | 69 ++- .../hadoop/hdfs/DFSStripedOutputStream.java | 501 +++++++++++------- .../java/org/apache/hadoop/hdfs/DFSUtil.java | 11 +- .../org/apache/hadoop/hdfs/DataStreamer.java | 15 +- .../hadoop/hdfs/StripedDataStreamer.java | 156 ++---- .../hdfs/server/namenode/FSDirectory.java | 2 +- .../apache/hadoop/hdfs/MiniDFSCluster.java | 2 - .../hdfs/TestDFSStripedOutputStream.java | 18 +- ...TestDFSStripedOutputStreamWithFailure.java | 323 +++++++++++ 10 files changed, 765 insertions(+), 335 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index a8df3f268f9a6..7efaa5a21cc07 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -172,3 +172,6 @@ HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via umamahesh) + + HDFS-7672. Handle write failure for stripping blocks and refactor the + existing code in DFSStripedOutputStream and StripedDataStreamer. (szetszwo) 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 0280d712a2084..8580357d8bfe7 100755 --- 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 @@ -24,6 +24,8 @@ import java.util.EnumSet; import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.crypto.CryptoProtocolVersion; @@ -86,6 +88,8 @@ @InterfaceAudience.Private public class DFSOutputStream extends FSOutputSummer implements Syncable, CanSetDropBehind { + static final Log LOG = LogFactory.getLog(DFSOutputStream.class); + /** * Number of times to retry creating a file when there are transient * errors (typically related to encryption zones and KeyProvider operations). @@ -419,24 +423,35 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, streamer.incBytesCurBlock(len); // If packet is full, enqueue it for transmission - // if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() || streamer.getBytesCurBlock() == blockSize) { - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("DFSClient writeChunk packet full seqno=" + - currentPacket.getSeqno() + - ", src=" + src + - ", bytesCurBlock=" + streamer.getBytesCurBlock() + - ", blockSize=" + blockSize + - ", appendChunk=" + streamer.getAppendChunk()); - } - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + enqueueCurrentPacketFull(); + } + } - adjustChunkBoundary(); + void enqueueCurrentPacket() throws IOException { + streamer.waitAndQueuePacket(currentPacket); + currentPacket = null; + } - endBlock(); + void enqueueCurrentPacketFull() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("enqueue full " + currentPacket + ", src=" + src + + ", bytesCurBlock=" + streamer.getBytesCurBlock() + + ", blockSize=" + blockSize + + ", appendChunk=" + streamer.getAppendChunk() + + ", " + streamer); } + enqueueCurrentPacket(); + adjustChunkBoundary(); + endBlock(); + } + + /** create an empty packet to mark the end of the block */ + void setCurrentPacket2Empty() throws InterruptedIOException { + currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), + streamer.getAndIncCurrentSeqno(), true); + currentPacket.setSyncBlock(shouldSyncBlock); } /** @@ -444,7 +459,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, * write filled up its partial chunk. Tell the summer to generate full * crc chunks from now on. */ - protected void adjustChunkBoundary() { + private void adjustChunkBoundary() { if (streamer.getAppendChunk() && streamer.getBytesCurBlock() % bytesPerChecksum == 0) { streamer.setAppendChunk(false); @@ -466,11 +481,8 @@ protected void adjustChunkBoundary() { */ protected void endBlock() throws IOException { if (streamer.getBytesCurBlock() == blockSize) { - currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), - streamer.getAndIncCurrentSeqno(), true); - currentPacket.setSyncBlock(shouldSyncBlock); - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + setCurrentPacket2Empty(); + enqueueCurrentPacket(); streamer.setBytesCurBlock(0); lastFlushOffset = 0; } @@ -592,8 +604,7 @@ private void flushOrSync(boolean isSync, EnumSet syncFlags) } if (currentPacket != null) { currentPacket.setSyncBlock(isSync); - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + enqueueCurrentPacket(); } if (endBlock && streamer.getBytesCurBlock() > 0) { // Need to end the current block, thus send an empty packet to @@ -601,8 +612,7 @@ private void flushOrSync(boolean isSync, EnumSet syncFlags) currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), streamer.getAndIncCurrentSeqno(), true); currentPacket.setSyncBlock(shouldSyncBlock || isSync); - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + enqueueCurrentPacket(); streamer.setBytesCurBlock(0); lastFlushOffset = 0; } else { @@ -779,15 +789,11 @@ protected synchronized void closeImpl() throws IOException { flushBuffer(); // flush from all upper layers if (currentPacket != null) { - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + enqueueCurrentPacket(); } if (streamer.getBytesCurBlock() != 0) { - // send an empty packet to mark the end of the block - currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), - streamer.getAndIncCurrentSeqno(), true); - currentPacket.setSyncBlock(shouldSyncBlock); + setCurrentPacket2Empty(); } flushInternal(); // flush all data to Datanodes @@ -901,4 +907,9 @@ ExtendedBlock getBlock() { public long getFileId() { return fileId; } + + @Override + public String toString() { + return getClass().getSimpleName() + ":" + streamer; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 71cdbb9d59add..bbc8ba0beba69 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -28,14 +28,16 @@ import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; -import org.apache.hadoop.hdfs.protocol.ECInfo; 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.io.MultipleIOException; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.util.DataChecksum; @@ -44,6 +46,8 @@ import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; +import com.google.common.base.Preconditions; + /**************************************************************** * The DFSStripedOutputStream class supports writing files in striped @@ -55,33 +59,154 @@ @InterfaceAudience.Private public class DFSStripedOutputStream extends DFSOutputStream { + /** Coordinate the communication between the streamers. */ + static class Coordinator { + private final List> endBlocks; + private final List> stripedBlocks; + private volatile boolean shouldLocateFollowingBlock = false; + + Coordinator(final int numDataBlocks, final int numAllBlocks) { + endBlocks = new ArrayList<>(numDataBlocks); + for (int i = 0; i < numDataBlocks; i++) { + endBlocks.add(new LinkedBlockingQueue(1)); + } - private final List streamers; - /** - * Size of each striping cell, must be a multiple of bytesPerChecksum - */ - private final ECInfo ecInfo; - private final int cellSize; - // checksum buffer, we only need to calculate checksum for parity blocks - private byte[] checksumBuf; - private ByteBuffer[] cellBuffers; + stripedBlocks = new ArrayList<>(numAllBlocks); + for (int i = 0; i < numAllBlocks; i++) { + stripedBlocks.add(new LinkedBlockingQueue(1)); + } + } - private final short numAllBlocks; - private final short numDataBlocks; + boolean shouldLocateFollowingBlock() { + return shouldLocateFollowingBlock; + } - private int curIdx = 0; - /* bytes written in current block group */ - //private long currentBlockGroupBytes = 0; + void putEndBlock(int i, ExtendedBlock block) { + shouldLocateFollowingBlock = true; - //TODO: Use ErasureCoder interface (HDFS-7781) - private RawErasureEncoder encoder; + final boolean b = endBlocks.get(i).offer(block); + Preconditions.checkState(b, "Failed to add " + block + + " to endBlocks queue, i=" + i); + } - private StripedDataStreamer getLeadingStreamer() { - return streamers.get(0); + ExtendedBlock getEndBlock(int i) throws InterruptedIOException { + try { + return endBlocks.get(i).poll(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw DFSUtil.toInterruptedIOException( + "getEndBlock interrupted, i=" + i, e); + } + } + + void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) { + ExtendedBlock b = endBlocks.get(i).peek(); + if (b == null) { + // streamer just has failed, put end block and continue + b = block; + putEndBlock(i, b); + } + b.setNumBytes(newBytes); + } + + void putStripedBlock(int i, LocatedBlock block) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("putStripedBlock " + block + ", i=" + i); + } + final boolean b = stripedBlocks.get(i).offer(block); + if (!b) { + throw new IOException("Failed: " + block + ", i=" + i); + } + } + + LocatedBlock getStripedBlock(int i) throws IOException { + final LocatedBlock lb; + try { + lb = stripedBlocks.get(i).poll(90, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e); + } + + if (lb == null) { + throw new IOException("Failed: i=" + i); + } + return lb; + } } - private long getBlockGroupSize() { - return blockSize * numDataBlocks; + /** Buffers for writing the data and parity cells of a strip. */ + class CellBuffers { + private final ByteBuffer[] buffers; + private final byte[][] checksumArrays; + + CellBuffers(int numParityBlocks) throws InterruptedException{ + if (cellSize % bytesPerChecksum != 0) { + throw new HadoopIllegalArgumentException("Invalid values: " + + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + + bytesPerChecksum + ") must divide cell size (=" + cellSize + ")."); + } + + checksumArrays = new byte[numParityBlocks][]; + final int size = getChecksumSize() * (cellSize / bytesPerChecksum); + for (int i = 0; i < checksumArrays.length; i++) { + checksumArrays[i] = new byte[size]; + } + + buffers = new ByteBuffer[numAllBlocks]; + for (int i = 0; i < buffers.length; i++) { + buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize)); + } + } + + private ByteBuffer[] getBuffers() { + return buffers; + } + + byte[] getChecksumArray(int i) { + return checksumArrays[i - numDataBlocks]; + } + + private int addTo(int i, byte[] b, int off, int len) { + final ByteBuffer buf = buffers[i]; + final int pos = buf.position() + len; + Preconditions.checkState(pos <= cellSize); + buf.put(b, off, len); + return pos; + } + + private void clear() { + for (int i = 0; i< numAllBlocks; i++) { + buffers[i].clear(); + if (i >= numDataBlocks) { + Arrays.fill(buffers[i].array(), (byte) 0); + } + } + } + + private void release() { + for (int i = 0; i < numAllBlocks; i++) { + byteArrayManager.release(buffers[i].array()); + } + } + + private void flipDataBuffers() { + for (int i = 0; i < numDataBlocks; i++) { + buffers[i].flip(); + } + } + } + + private final Coordinator coordinator; + private final CellBuffers cellBuffers; + private final RawErasureEncoder encoder; + private final List streamers; + + /** Size of each striping cell, must be a multiple of bytesPerChecksum */ + private final int cellSize; + private final int numAllBlocks; + private final int numDataBlocks; + + private StripedDataStreamer getLeadingStreamer() { + return streamers.get(0); } /** Construct a new output stream for creating a file. */ @@ -90,82 +215,94 @@ private long getBlockGroupSize() { DataChecksum checksum, String[] favoredNodes) throws IOException { super(dfsClient, src, stat, flag, progress, checksum, favoredNodes); - DFSClient.LOG.info("Creating striped output stream"); + if (LOG.isDebugEnabled()) { + LOG.debug("Creating DFSStripedOutputStream for " + src); + } // ECInfo is restored from NN just before writing striped files. - ecInfo = dfsClient.getErasureCodingInfo(src); - cellSize = ecInfo.getSchema().getChunkSize(); - numAllBlocks = (short)(ecInfo.getSchema().getNumDataUnits() - + ecInfo.getSchema().getNumParityUnits()); - numDataBlocks = (short)ecInfo.getSchema().getNumDataUnits(); + //TODO reduce an rpc call HDFS-8289 + final ECSchema schema = dfsClient.getErasureCodingInfo(src).getSchema(); + final int numParityBlocks = schema.getNumParityUnits(); + cellSize = schema.getChunkSize(); + numDataBlocks = schema.getNumDataUnits(); + numAllBlocks = numDataBlocks + numParityBlocks; - checkConfiguration(); - - checksumBuf = new byte[getChecksumSize() * (cellSize / bytesPerChecksum)]; - cellBuffers = new ByteBuffer[numAllBlocks]; - List> stripeBlocks = new ArrayList<>(); + encoder = new RSRawEncoder(); + encoder.initialize(numDataBlocks, numParityBlocks, cellSize); - for (int i = 0; i < numAllBlocks; i++) { - stripeBlocks.add(new LinkedBlockingQueue(numAllBlocks)); - try { - cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize)); - } catch (InterruptedException ie) { - final InterruptedIOException iioe = new InterruptedIOException( - "create cell buffers"); - iioe.initCause(ie); - throw iioe; - } + coordinator = new Coordinator(numDataBlocks, numAllBlocks); + try { + cellBuffers = new CellBuffers(numParityBlocks); + } catch (InterruptedException ie) { + throw DFSUtil.toInterruptedIOException("Failed to create cell buffers", ie); } - encoder = new RSRawEncoder(); - encoder.initialize(numDataBlocks, - numAllBlocks - numDataBlocks, cellSize); List s = new ArrayList<>(numAllBlocks); for (short i = 0; i < numAllBlocks; i++) { - StripedDataStreamer streamer = new StripedDataStreamer(stat, null, + StripedDataStreamer streamer = new StripedDataStreamer(stat, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager, - i, stripeBlocks, favoredNodes); + favoredNodes, i, coordinator); s.add(streamer); } streamers = Collections.unmodifiableList(s); + setCurrentStreamer(0); + } - refreshStreamer(); + StripedDataStreamer getStripedDataStreamer(int i) { + return streamers.get(i); } - private void checkConfiguration() { - if (cellSize % bytesPerChecksum != 0) { - throw new HadoopIllegalArgumentException("Invalid values: " - + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum - + ") must divide cell size (=" + cellSize + ")."); - } + int getCurrentIndex() { + return getCurrentStreamer().getIndex(); } - private void refreshStreamer() { - streamer = streamers.get(curIdx); + StripedDataStreamer getCurrentStreamer() { + return (StripedDataStreamer)streamer; } - private void moveToNextStreamer() { - curIdx = (curIdx + 1) % numAllBlocks; - refreshStreamer(); + private StripedDataStreamer setCurrentStreamer(int i) { + streamer = streamers.get(i); + return getCurrentStreamer(); } /** - * encode the buffers. - * After encoding, flip each buffer. + * Encode the buffers, i.e. compute parities. * * @param buffers data buffers + parity buffers */ - private void encode(ByteBuffer[] buffers) { - ByteBuffer[] dataBuffers = new ByteBuffer[numDataBlocks]; - ByteBuffer[] parityBuffers = new ByteBuffer[numAllBlocks - numDataBlocks]; - for (int i = 0; i < numAllBlocks; i++) { - if (i < numDataBlocks) { - dataBuffers[i] = buffers[i]; - } else { - parityBuffers[i - numDataBlocks] = buffers[i]; + private static void encode(RawErasureEncoder encoder, int numData, + ByteBuffer[] buffers) { + final ByteBuffer[] dataBuffers = new ByteBuffer[numData]; + final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData]; + System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length); + System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length); + + encoder.encode(dataBuffers, parityBuffers); + } + + + private void checkStreamers() throws IOException { + int count = 0; + for(StripedDataStreamer s : streamers) { + if (!s.isFailed()) { + count++; } } - encoder.encode(dataBuffers, parityBuffers); + if (LOG.isDebugEnabled()) { + LOG.debug("checkStreamers: " + streamers); + LOG.debug("count=" + count); + } + if (count < numDataBlocks) { + throw new IOException("Failed: the number of remaining blocks = " + + count + " < the number of data blocks = " + numDataBlocks); + } + } + + private void handleStreamerFailure(String err, Exception e) throws IOException { + LOG.warn("Failed: " + err + ", " + this, e); + getCurrentStreamer().setIsFailed(true); + checkStreamers(); + currentPacket = null; } /** @@ -173,11 +310,12 @@ private void encode(ByteBuffer[] buffers) { * writing parity blocks. * * @param byteBuffer the given buffer to generate packets + * @param checksumBuf the checksum buffer * @return packets generated * @throws IOException */ - private List generatePackets(ByteBuffer byteBuffer) - throws IOException{ + private List generatePackets( + ByteBuffer byteBuffer, byte[] checksumBuf) throws IOException{ List packets = new ArrayList<>(); assert byteBuffer.hasArray(); getDataChecksum().calculateChunkedSums(byteBuffer.array(), 0, @@ -201,82 +339,47 @@ private List generatePackets(ByteBuffer byteBuffer) } @Override - protected synchronized void writeChunk(byte[] b, int offset, int len, + protected synchronized void writeChunk(byte[] bytes, int offset, int len, byte[] checksum, int ckoff, int cklen) throws IOException { - super.writeChunk(b, offset, len, checksum, ckoff, cklen); - - if (getSizeOfCellnBuffer(curIdx) <= cellSize) { - addToCellBuffer(b, offset, len); - } else { - String msg = "Writing a chunk should not overflow the cell buffer."; - DFSClient.LOG.info(msg); - throw new IOException(msg); - } - - // If current packet has not been enqueued for transmission, - // but the cell buffer is full, we need to enqueue the packet - if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) { - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("DFSClient writeChunk cell buffer full seqno=" + - currentPacket.getSeqno() + - ", curIdx=" + curIdx + - ", src=" + src + - ", bytesCurBlock=" + streamer.getBytesCurBlock() + - ", blockSize=" + blockSize + - ", appendChunk=" + streamer.getAppendChunk()); + final int index = getCurrentIndex(); + final StripedDataStreamer current = getCurrentStreamer(); + final int pos = cellBuffers.addTo(index, bytes, offset, len); + final boolean cellFull = pos == cellSize; + + final long oldBytes = current.getBytesCurBlock(); + if (!current.isFailed()) { + try { + super.writeChunk(bytes, offset, len, checksum, ckoff, cklen); + + // cell is full and current packet has not been enqueued, + if (cellFull && currentPacket != null) { + enqueueCurrentPacketFull(); + } + } catch(Exception e) { + handleStreamerFailure("offset=" + offset + ", length=" + len, e); } - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; - adjustChunkBoundary(); - endBlock(); + } + + if (current.isFailed()) { + final long newBytes = oldBytes + len; + coordinator.setBytesEndBlock(index, newBytes, current.getBlock()); + current.setBytesCurBlock(newBytes); } // Two extra steps are needed when a striping cell is full: // 1. Forward the current index pointer // 2. Generate parity packets if a full stripe of data cells are present - if (getSizeOfCellnBuffer(curIdx) == cellSize) { - //move curIdx to next cell - moveToNextStreamer(); + if (cellFull) { + int next = index + 1; //When all data cells in a stripe are ready, we need to encode //them and generate some parity cells. These cells will be //converted to packets and put to their DataStreamer's queue. - if (curIdx == numDataBlocks) { - //encode the data cells - for (int k = 0; k < numDataBlocks; k++) { - cellBuffers[k].flip(); - } - encode(cellBuffers); - for (int i = numDataBlocks; i < numAllBlocks; i++) { - ByteBuffer parityBuffer = cellBuffers[i]; - List packets = generatePackets(parityBuffer); - for (DFSPacket p : packets) { - currentPacket = p; - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; - } - endBlock(); - moveToNextStreamer(); - } - //read next stripe to cellBuffers - clearCellBuffers(); - } - } - } - - private void addToCellBuffer(byte[] b, int off, int len) { - cellBuffers[curIdx].put(b, off, len); - } - - private int getSizeOfCellnBuffer(int cellIndex) { - return cellBuffers[cellIndex].position(); - } - - private void clearCellBuffers() { - for (int i = 0; i< numAllBlocks; i++) { - cellBuffers[i].clear(); - if (i >= numDataBlocks) { - Arrays.fill(cellBuffers[i].array(), (byte) 0); + if (next == numDataBlocks) { + cellBuffers.flipDataBuffers(); + writeParityCells(); + next = 0; } + setCurrentStreamer(next); } } @@ -284,20 +387,14 @@ private int stripeDataSize() { return numDataBlocks * cellSize; } - private void notSupported(String headMsg) - throws IOException{ - throw new IOException( - headMsg + " is now not supported for striping layout."); - } - @Override - public void hflush() throws IOException { - notSupported("hflush"); + public void hflush() { + throw new UnsupportedOperationException(); } @Override - public void hsync() throws IOException { - notSupported("hsync"); + public void hsync() { + throw new UnsupportedOperationException(); } @Override @@ -327,29 +424,28 @@ boolean isClosed() { return closed || getLeadingStreamer().streamerClosed(); } - // shutdown datastreamer and responseprocessor threads. - // interrupt datastreamer if force is true @Override protected void closeThreads(boolean force) throws IOException { - int index = 0; - boolean exceptionOccurred = false; + final MultipleIOException.Builder b = new MultipleIOException.Builder(); for (StripedDataStreamer streamer : streamers) { try { streamer.close(force); streamer.join(); streamer.closeSocket(); - } catch (InterruptedException | IOException e) { - DFSClient.LOG.error("Failed to shutdown streamer: name=" - + streamer.getName() + ", index=" + index + ", file=" + src, e); - exceptionOccurred = true; + } catch(Exception e) { + try { + handleStreamerFailure("force=" + force, e); + } catch(IOException ioe) { + b.add(ioe); + } } finally { streamer.setSocketToNull(); setClosed(); - index++; } } - if (exceptionOccurred) { - throw new IOException("Failed to shutdown streamer"); + final IOException ioe = b.build(); + if (ioe != null) { + throw ioe; } } @@ -370,50 +466,69 @@ private void writeParityCellsForLastStripe() throws IOException { if (currentBlockGroupBytes % stripeDataSize() == 0) { return; } - long firstCellSize = getLeadingStreamer().getBytesCurBlock() % cellSize; - long parityCellSize = firstCellSize > 0 && firstCellSize < cellSize ? + + final int firstCellSize = (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize); + final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize? firstCellSize : cellSize; + final ByteBuffer[] buffers = cellBuffers.getBuffers(); for (int i = 0; i < numAllBlocks; i++) { // Pad zero bytes to make all cells exactly the size of parityCellSize // If internal block is smaller than parity block, pad zero bytes. // Also pad zero bytes to all parity cells - int position = cellBuffers[i].position(); + final int position = buffers[i].position(); assert position <= parityCellSize : "If an internal block is smaller" + " than parity block, then its last cell should be small than last" + " parity cell"; for (int j = 0; j < parityCellSize - position; j++) { - cellBuffers[i].put((byte) 0); + buffers[i].put((byte) 0); } - cellBuffers[i].flip(); + buffers[i].flip(); } - encode(cellBuffers); - // write parity cells - curIdx = numDataBlocks; - refreshStreamer(); + writeParityCells(); + } + + void writeParityCells() throws IOException { + final ByteBuffer[] buffers = cellBuffers.getBuffers(); + //encode the data cells + encode(encoder, numDataBlocks, buffers); for (int i = numDataBlocks; i < numAllBlocks; i++) { - ByteBuffer parityBuffer = cellBuffers[i]; - List packets = generatePackets(parityBuffer); - for (DFSPacket p : packets) { - currentPacket = p; - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + writeParity(i, buffers[i], cellBuffers.getChecksumArray(i)); + } + cellBuffers.clear(); + } + + void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf + ) throws IOException { + final StripedDataStreamer current = setCurrentStreamer(index); + final int len = buffer.limit(); + + final long oldBytes = current.getBytesCurBlock(); + if (!current.isFailed()) { + try { + for (DFSPacket p : generatePackets(buffer, checksumBuf)) { + streamer.waitAndQueuePacket(p); + } + endBlock(); + } catch(Exception e) { + handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e); } - endBlock(); - moveToNextStreamer(); } - clearCellBuffers(); + if (current.isFailed()) { + final long newBytes = oldBytes + len; + current.setBytesCurBlock(newBytes); + } } @Override void setClosed() { super.setClosed(); for (int i = 0; i < numAllBlocks; i++) { - byteArrayManager.release(cellBuffers[i].array()); streamers.get(i).release(); } + cellBuffers.release(); } @Override @@ -425,25 +540,31 @@ protected synchronized void closeImpl() throws IOException { try { // flush from all upper layers - flushBuffer(); - if (currentPacket != null) { - streamer.waitAndQueuePacket(currentPacket); - currentPacket = null; + try { + flushBuffer(); + if (currentPacket != null) { + enqueueCurrentPacket(); + } + } catch(Exception e) { + handleStreamerFailure("closeImpl", e); } + // if the last stripe is incomplete, generate and write parity cells writeParityCellsForLastStripe(); for (int i = 0; i < numAllBlocks; i++) { - curIdx = i; - refreshStreamer(); - if (streamer.getBytesCurBlock() > 0) { - // send an empty packet to mark the end of the block - currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(), - streamer.getAndIncCurrentSeqno(), true); - currentPacket.setSyncBlock(shouldSyncBlock); + final StripedDataStreamer s = setCurrentStreamer(i); + if (!s.isFailed()) { + try { + if (s.getBytesCurBlock() > 0) { + setCurrentPacket2Empty(); + } + // flush all data to Datanode + flushInternal(); + } catch(Exception e) { + handleStreamerFailure("closeImpl", e); + } } - // flush all data to Datanode - flushInternal(); } closeThreads(false); 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 cae56c04da512..2e2ecfd652a21 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 @@ -36,6 +36,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY; import java.io.IOException; +import java.io.InterruptedIOException; import java.io.PrintStream; import java.io.UnsupportedEncodingException; import java.net.InetAddress; @@ -55,7 +56,6 @@ import javax.net.SocketFactory; -import com.google.common.collect.Sets; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.Option; @@ -96,6 +96,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.google.protobuf.BlockingService; @InterfaceAudience.Private @@ -1513,7 +1514,7 @@ public static int getIoFileBufferSize(Configuration conf) { public static int getSmallBufferSize(Configuration conf) { return Math.min(getIoFileBufferSize(conf) / 2, 512); } - + /** * Probe for HDFS Encryption being enabled; this uses the value of * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI}, @@ -1527,4 +1528,10 @@ public static boolean isHDFSEncryptionEnabled(Configuration conf) { DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty(); } + public static InterruptedIOException toInterruptedIOException(String message, + InterruptedException e) { + final InterruptedIOException iioe = new InterruptedIOException(message); + iioe.initCause(e); + return iioe; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index 631f3869a299d..8f07341f749c2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -575,7 +575,7 @@ public void run() { // get new block from namenode. if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) { if(LOG.isDebugEnabled()) { - LOG.debug("Allocating new block"); + LOG.debug("Allocating new block " + this); } setPipeline(nextBlockOutputStream()); initDataStreaming(); @@ -593,10 +593,7 @@ public void run() { long lastByteOffsetInBlock = one.getLastByteOffsetBlock(); if (lastByteOffsetInBlock > stat.getBlockSize()) { throw new IOException("BlockSize " + stat.getBlockSize() + - " is smaller than data size. " + - " Offset of packet in block " + - lastByteOffsetInBlock + - " Aborting file " + src); + " < lastByteOffsetInBlock, " + this + ", " + one); } if (one.isLastPacketInBlock()) { @@ -1751,7 +1748,7 @@ void queuePacket(DFSPacket packet) { dataQueue.addLast(packet); lastQueuedSeqno = packet.getSeqno(); if (LOG.isDebugEnabled()) { - LOG.debug("Queued packet " + packet.getSeqno()); + LOG.debug("Queued " + packet + ", " + this); } dataQueue.notifyAll(); } @@ -1901,4 +1898,10 @@ void closeSocket() throws IOException { s.close(); } } + + @Override + public String toString() { + return (block == null? null: block.getLocalBlock()) + + "@" + Arrays.toString(getNodes()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index ef7e2a687a6b9..258fc6505dd85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -18,8 +18,14 @@ package org.apache.hadoop.hdfs; -import java.util.List; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -31,15 +37,6 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; -import java.io.IOException; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; - /**************************************************************************** * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}. * There are two kinds of StripedDataStreamer, leading streamer and ordinary @@ -49,40 +46,32 @@ * ****************************************************************************/ public class StripedDataStreamer extends DataStreamer { - private final short index; - private final List> stripedBlocks; - private boolean hasCommittedBlock = false; + private final Coordinator coordinator; + private final int index; + private volatile boolean isFailed; - StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block, + StripedDataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, - ByteArrayManager byteArrayManage, short index, - List> stripedBlocks, - String[] favoredNodes) { - super(stat, block, dfsClient, src, progress, checksum, cachingStrategy, + ByteArrayManager byteArrayManage, String[] favoredNodes, + short index, Coordinator coordinator) { + super(stat, null, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManage, favoredNodes); this.index = index; - this.stripedBlocks = stripedBlocks; + this.coordinator = coordinator; } - /** - * Construct a data streamer for appending to the last partial block - * @param lastBlock last block of the file to be appended - * @param stat status of the file to be appended - * @throws IOException if error occurs - */ - StripedDataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, - DFSClient dfsClient, String src, - Progressable progress, DataChecksum checksum, - AtomicReference cachingStrategy, - ByteArrayManager byteArrayManage, short index, - List> stripedBlocks) - throws IOException { - super(lastBlock, stat, dfsClient, src, progress, checksum, cachingStrategy, - byteArrayManage); - this.index = index; - this.stripedBlocks = stripedBlocks; + int getIndex() { + return index; + } + + void setIsFailed(boolean isFailed) { + this.isFailed = isFailed; + } + + boolean isFailed() { + return isFailed; } public boolean isLeadingStreamer () { @@ -95,18 +84,8 @@ private boolean isParityStreamer() { @Override protected void endBlock() { - if (!isLeadingStreamer() && !isParityStreamer()) { - // before retrieving a new block, transfer the finished block to - // leading streamer - LocatedBlock finishedBlock = new LocatedBlock( - new ExtendedBlock(block.getBlockPoolId(), block.getBlockId(), - block.getNumBytes(), block.getGenerationStamp()), null); - try { - boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30, - TimeUnit.SECONDS); - } catch (InterruptedException ie) { - // TODO: Handle InterruptedException (HDFS-7786) - } + if (!isParityStreamer()) { + coordinator.putEndBlock(index, block); } super.endBlock(); } @@ -114,71 +93,40 @@ protected void endBlock() { @Override protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) throws IOException { - LocatedBlock lb = null; if (isLeadingStreamer()) { - if (hasCommittedBlock) { - /** - * when committing a block group, leading streamer has to adjust - * {@link block} to include the size of block group - */ - for (int i = 1; i < NUM_DATA_BLOCKS; i++) { - try { - LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30, - TimeUnit.SECONDS); - if (finishedLocatedBlock == null) { - throw new IOException("Fail to get finished LocatedBlock " + - "from streamer, i=" + i); - } - ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock(); - long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes(); - if (block != null) { - block.setNumBytes(block.getNumBytes() + bytes); - } - } catch (InterruptedException ie) { - DFSClient.LOG.info("InterruptedException received when putting" + - " a block to stripeBlocks, ie = " + ie); - } + if (coordinator.shouldLocateFollowingBlock()) { + // set numByte for the previous block group + long bytes = 0; + for (int i = 0; i < NUM_DATA_BLOCKS; i++) { + final ExtendedBlock b = coordinator.getEndBlock(i); + bytes += b == null ? 0 : b.getNumBytes(); } + block.setNumBytes(bytes); } - lb = super.locateFollowingBlock(excludedNodes); - hasCommittedBlock = true; - assert lb instanceof LocatedStripedBlock; - DFSClient.LOG.debug("Leading streamer obtained bg " + lb); - LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( - (LocatedStripedBlock) lb, BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, - NUM_PARITY_BLOCKS); + final LocatedStripedBlock lsb + = (LocatedStripedBlock)super.locateFollowingBlock(excludedNodes); + if (LOG.isDebugEnabled()) { + LOG.debug("Obtained block group " + lsb); + } + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(lsb, + BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) : "Fail to get block group from namenode: blockGroupSize: " + (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " + blocks.length; - lb = blocks[0]; - for (int i = 1; i < blocks.length; i++) { - try { - boolean offSuccess = stripedBlocks.get(i).offer(blocks[i], - 90, TimeUnit.SECONDS); - if(!offSuccess){ - String msg = "Fail to put block to stripeBlocks. i = " + i; - DFSClient.LOG.info(msg); - throw new IOException(msg); - } else { - DFSClient.LOG.info("Allocate a new block to a streamer. i = " + i - + ", block: " + blocks[i]); - } - } catch (InterruptedException ie) { - DFSClient.LOG.info("InterruptedException received when putting" + - " a block to stripeBlocks, ie = " + ie); - } - } - } else { - try { - // wait 90 seconds to get a block from the queue - lb = stripedBlocks.get(index).poll(90, TimeUnit.SECONDS); - } catch (InterruptedException ie) { - DFSClient.LOG.info("InterruptedException received when retrieving " + - "a block from stripeBlocks, ie = " + ie); + for (int i = 0; i < blocks.length; i++) { + coordinator.putStripedBlock(i, blocks[i]); } } - return lb; + + return coordinator.getStripedBlock(index); + } + + @Override + public String toString() { + return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0) + + ", " + super.toString(); } } 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 7392552133fd0..8f843d5cbb91b 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 @@ -390,7 +390,7 @@ void enableQuotaChecks() { void disableQuotaChecks() { skipQuotaCheck = true; } - + /** * This is a wrapper for resolvePath(). If the path passed * is prefixed with /.reserved/raw, then it checks to ensure that the caller 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 fdbacdca40f05..4ec9bf96273e1 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 @@ -1998,8 +1998,6 @@ public synchronized DataNodeProperties stopDataNode(String dnName) { int node = -1; for (int i = 0; i < dataNodes.size(); i++) { DataNode dn = dataNodes.get(i).datanode; - LOG.info("DN name=" + dnName + " found DN=" + dn + - " with name=" + dn.getDisplayName()); if (dnName.equals(dn.getDatanodeId().getXferAddr())) { node = i; break; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 5ce94ee1a41cc..ec98e68612f61 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -35,6 +35,8 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -42,6 +44,12 @@ public class TestDFSStripedOutputStream { public static final Log LOG = LogFactory.getLog(TestDFSStripedOutputStream.class); + + static { + GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); + GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); + } + private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; @@ -245,6 +253,11 @@ void checkData(String src, int writeBytes) throws IOException { static void verifyParity(final long size, final int cellSize, byte[][] dataBytes, byte[][] parityBytes) { + verifyParity(size, cellSize, dataBytes, parityBytes, -1); + } + + static void verifyParity(final long size, final int cellSize, + byte[][] dataBytes, byte[][] parityBytes, int killedDnIndex) { // verify the parity blocks int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength( size, cellSize, dataBytes.length, dataBytes.length); @@ -265,7 +278,10 @@ static void verifyParity(final long size, final int cellSize, encoder.initialize(dataBytes.length, parityBytes.length, cellSize); encoder.encode(dataBytes, expectedParityBytes); for (int i = 0; i < parityBytes.length; i++) { - Assert.assertArrayEquals(expectedParityBytes[i], parityBytes[i]); + if (i != killedDnIndex) { + Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + killedDnIndex, + expectedParityBytes[i], parityBytes[i]); + } } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java new file mode 100644 index 0000000000000..c2e588ad5012d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -0,0 +1,323 @@ +/** + * 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 java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.log4j.Level; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestDFSStripedOutputStreamWithFailure { + public static final Log LOG = LogFactory.getLog( + TestDFSStripedOutputStreamWithFailure.class); + static { + GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); + GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); + } + + private static final int NUM_DATA_BLOCKS = HdfsConstants.NUM_DATA_BLOCKS; + private static final int NUM_PARITY_BLOCKS = HdfsConstants.NUM_PARITY_BLOCKS; + private static final int CELL_SIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final int STRIPES_PER_BLOCK = 4; + private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK; + private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS; + + private final HdfsConfiguration conf = new HdfsConfiguration(); + private MiniDFSCluster cluster; + private DistributedFileSystem dfs; + private final Path dir = new Path("/" + + TestDFSStripedOutputStreamWithFailure.class.getSimpleName()); + + + @Before + public void setup() throws IOException { + final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + dfs = cluster.getFileSystem(); + dfs.mkdirs(dir); + dfs.createErasureCodingZone(dir, null); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + private static byte getByte(long pos) { + return (byte)pos; + } + + @Test(timeout=120000) + public void testDatanodeFailure1() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 1; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure2() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 2; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure3() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 3; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure4() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 4; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure5() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 5; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure6() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 6; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure7() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 7; + runTest("file" + dn, length, dn); + } + + @Test(timeout=120000) + public void testDatanodeFailure8() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 8; + runTest("file" + dn, length, dn); + } + + private void runTest(final String src, final int length, final int dnIndex) { + try { + cluster.startDataNodes(conf, 1, true, null, null); + cluster.waitActive(); + + runTest(new Path(dir, src), length, dnIndex); + } catch(Exception e) { + LOG.info("FAILED", e); + Assert.fail(StringUtils.stringifyException(e)); + } + } + + private void runTest(final Path p, final int length, + final int dnIndex) throws Exception { + LOG.info("p=" + p + ", length=" + length + ", dnIndex=" + dnIndex); + final String fullPath = p.toString(); + + final AtomicInteger pos = new AtomicInteger(); + final FSDataOutputStream out = dfs.create(p); + final AtomicBoolean killed = new AtomicBoolean(); + final Thread killer = new Thread(new Runnable() { + @Override + public void run() { + killDatanode(cluster, (DFSStripedOutputStream)out.getWrappedStream(), + dnIndex, pos); + killed.set(true); + } + }); + killer.start(); + + final int mask = (1 << 16) - 1; + for(; pos.get() < length; ) { + final int i = pos.getAndIncrement(); + write(out, i); + if ((i & mask) == 0) { + final long ms = 100; + LOG.info("i=" + i + " sleep " + ms); + Thread.sleep(ms); + } + } + killer.join(10000); + Assert.assertTrue(killed.get()); + out.close(); + + // check file length + final FileStatus status = dfs.getFileStatus(p); + Assert.assertEquals(length, status.getLen()); + + checkData(dfs, fullPath, length, dnIndex); + } + + static void write(FSDataOutputStream out, int i) throws IOException { + try { + out.write(getByte(i)); + } catch(IOException ioe) { + throw new IOException("Failed at i=" + i, ioe); + } + } + + static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { + for(;;) { + final DatanodeInfo[] datanodes = streamer.getNodes(); + if (datanodes != null) { + Assert.assertEquals(1, datanodes.length); + Assert.assertNotNull(datanodes[0]); + return datanodes[0]; + } + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + return null; + } + } + } + + static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, + final int dnIndex, final AtomicInteger pos) { + final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex); + final DatanodeInfo datanode = getDatanodes(s); + LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos); + cluster.stopDataNode(datanode.getXferAddr()); + } + + static void checkData(DistributedFileSystem dfs, String src, int length, + int killedDnIndex) throws IOException { + List> blockGroupList = new ArrayList<>(); + LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L); + final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1; + Assert.assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size()); + + for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { + Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) firstBlock, + CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + blockGroupList.add(Arrays.asList(blocks)); + } + + // test each block group + for (int group = 0; group < blockGroupList.size(); group++) { + final boolean isLastGroup = group == blockGroupList.size() - 1; + final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE + : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE; + final int numCellInGroup = (int)((groupSize - 1)/CELL_SIZE + 1); + final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS; + final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE; + + //get the data of this block + List blockList = blockGroupList.get(group); + byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][]; + byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][]; + + // for each block, use BlockReader to read data + for (int i = 0; i < blockList.size(); i++) { + final int j = i >= NUM_DATA_BLOCKS? 0: i; + final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS + + (j <= lastCellIndex? 1: 0); + final int blockSize = numCellInBlock*CELL_SIZE + + (isLastGroup && i == lastCellIndex? lastCellSize - CELL_SIZE: 0); + + final byte[] blockBytes = new byte[blockSize]; + if (i < NUM_DATA_BLOCKS) { + dataBlockBytes[i] = blockBytes; + } else { + parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes; + } + + final LocatedBlock lb = blockList.get(i); + LOG.info("XXX i=" + i + ", lb=" + lb); + if (lb == null) { + continue; + } + final ExtendedBlock block = lb.getBlock(); + Assert.assertEquals(blockSize, block.getNumBytes()); + + + if (block.getNumBytes() == 0) { + continue; + } + + if (i != killedDnIndex) { + final BlockReader blockReader = BlockReaderTestUtil.getBlockReader( + dfs, lb, 0, block.getNumBytes()); + blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); + blockReader.close(); + } + } + + // check data + final int groupPosInFile = group*BLOCK_GROUP_SIZE; + for (int i = 0; i < dataBlockBytes.length; i++) { + final byte[] actual = dataBlockBytes[i]; + for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) { + final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG( + CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile; + Assert.assertTrue(posInFile < length); + final byte expected = getByte(posInFile); + + if (i == killedDnIndex) { + actual[posInBlk] = expected; + } else { + String s = "expected=" + expected + " but actual=" + actual[posInBlk] + + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk + + ". group=" + group + ", i=" + i; + Assert.assertEquals(s, expected, actual[posInBlk]); + } + } + } + + // check parity + TestDFSStripedOutputStream.verifyParity( + lbs.getLocatedBlocks().get(group).getBlockSize(), + CELL_SIZE, dataBlockBytes, parityBlockBytes, + killedDnIndex - dataBlockBytes.length); + } + } +} From 6616de24cb14f1c2d0d6568fd4382062618834bd Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 5 May 2015 16:33:56 -0700 Subject: [PATCH 100/212] HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. Contributed by Yi Liu. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../org/apache/hadoop/hdfs/BlockReader.java | 6 + .../apache/hadoop/hdfs/BlockReaderLocal.java | 5 + .../hadoop/hdfs/BlockReaderLocalLegacy.java | 5 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 6 + .../org/apache/hadoop/hdfs/DFSPacket.java | 10 +- .../apache/hadoop/hdfs/RemoteBlockReader.java | 5 + .../hadoop/hdfs/RemoteBlockReader2.java | 5 + .../hadoop/hdfs/server/datanode/DNConf.java | 27 + .../hadoop/hdfs/server/datanode/DataNode.java | 31 +- .../erasurecode/ErasureCodingWorker.java | 893 +++++++++++++++++- .../hadoop/hdfs/util/StripedBlockUtil.java | 49 +- .../src/main/resources/hdfs-default.xml | 31 +- .../hadoop/hdfs/TestRecoverStripedFile.java | 356 +++++++ 14 files changed, 1377 insertions(+), 55 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 7efaa5a21cc07..0d2d4485e1647 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -175,3 +175,6 @@ HDFS-7672. Handle write failure for stripping blocks and refactor the existing code in DFSStripedOutputStream and StripedDataStreamer. (szetszwo) + + HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. + (Yi Liu via Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java index aa3e8ba990906..0a5511e16f0da 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.ByteBufferReadable; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.hdfs.shortcircuit.ClientMmap; +import org.apache.hadoop.util.DataChecksum; /** * A BlockReader is responsible for reading a single block @@ -99,4 +100,9 @@ public interface BlockReader extends ByteBufferReadable { * supported. */ ClientMmap getClientMmap(EnumSet opts); + + /** + * @return The DataChecksum used by the read block + */ + DataChecksum getDataChecksum(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java index d913f3a283513..0b2420d545156 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java @@ -738,4 +738,9 @@ void forceAnchorable() { void forceUnanchorable() { replica.getSlot().makeUnanchorable(); } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java index c16ffdf2e0e11..04cf733cc0157 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java @@ -732,4 +732,9 @@ public boolean isShortCircuit() { public ClientMmap getClientMmap(EnumSet opts) { return null; } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 6bc005bd14b9f..d5d30955b5b85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -369,6 +369,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT = 21600; public static final String DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY = "dfs.datanode.directoryscan.threads"; public static final int DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT = 1; + public static final String DFS_DATANODE_STRIPED_READ_THREADS_KEY = "dfs.datanode.stripedread.threads"; + public static final int DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20; + public static final String DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size"; + public static final int DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 256 * 1024; + public static final String DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY = "dfs.datanode.stripedread.threshold.millis"; + public static final int DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s public static final String DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface"; public static final String DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default"; public static final String DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java index 9cd1ec1fcc33e..a26e35e7c586b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java @@ -37,7 +37,7 @@ ****************************************************************/ @InterfaceAudience.Private -class DFSPacket { +public class DFSPacket { public static final long HEART_BEAT_SEQNO = -1L; private static long[] EMPTY = new long[0]; private final long seqno; // sequence number of buffer in block @@ -80,7 +80,7 @@ class DFSPacket { * @param checksumSize the size of checksum * @param lastPacketInBlock if this is the last packet */ - DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno, + public DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno, int checksumSize, boolean lastPacketInBlock) { this.lastPacketInBlock = lastPacketInBlock; this.numChunks = 0; @@ -114,7 +114,7 @@ synchronized void writeData(byte[] inarray, int off, int len) dataPos += len; } - synchronized void writeData(ByteBuffer inBuffer, int len) + public synchronized void writeData(ByteBuffer inBuffer, int len) throws ClosedChannelException { checkBuffer(); len = len > inBuffer.remaining() ? inBuffer.remaining() : len; @@ -135,7 +135,7 @@ synchronized void writeData(ByteBuffer inBuffer, int len) * @param len the length of checksums to write * @throws ClosedChannelException */ - synchronized void writeChecksum(byte[] inarray, int off, int len) + public synchronized void writeChecksum(byte[] inarray, int off, int len) throws ClosedChannelException { checkBuffer(); if (len == 0) { @@ -154,7 +154,7 @@ synchronized void writeChecksum(byte[] inarray, int off, int len) * @param stm * @throws IOException */ - synchronized void writeTo(DataOutputStream stm) throws IOException { + public synchronized void writeTo(DataOutputStream stm) throws IOException { checkBuffer(); final int dataLen = dataPos - dataStart; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index d70f41904bc68..70cce7e2e5a4b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -505,4 +505,9 @@ public boolean isShortCircuit() { public ClientMmap getClientMmap(EnumSet opts) { return null; } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index c368d6515f1e1..cce44b7f77722 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -474,4 +474,9 @@ public boolean isShortCircuit() { public ClientMmap getClientMmap(EnumSet opts) { return null; } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java index 4b7fbc3c1f890..d25642f047a6f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java @@ -235,6 +235,33 @@ public long getMaxLockedMemory() { return maxLockedMemory; } + /** + * Returns true if connect to datanode via hostname + * + * @return boolean true if connect to datanode via hostname + */ + public boolean getConnectToDnViaHostname() { + return connectToDnViaHostname; + } + + /** + * Returns socket timeout + * + * @return int socket timeout + */ + public int getSocketTimeout() { + return socketTimeout; + } + + /** + * Returns socket write timeout + * + * @return int socket write timeout + */ + public int getSocketWriteTimeout() { + return socketWriteTimeout; + } + /** * Returns the SaslPropertiesResolver configured for use with * DataTransferProtocol, or null if not configured. 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 221ba386ac190..5eca2c7e98650 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 @@ -1161,7 +1161,8 @@ void startDataNode(Configuration conf, saslClient = new SaslDataTransferClient(dnConf.conf, dnConf.saslPropsResolver, dnConf.trustedChannelResolver); saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager); - ecWorker = new ErasureCodingWorker(conf); // Initialize ErasureCoding worker + // Initialize ErasureCoding worker + ecWorker = new ErasureCodingWorker(conf, this); } /** @@ -1226,6 +1227,10 @@ public static String generateUuid() { return UUID.randomUUID().toString(); } + public SaslDataTransferClient getSaslClient() { + return saslClient; + } + /** * 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. @@ -1488,7 +1493,7 @@ public DatanodeRegistration getDNRegistrationForBP(String bpid) /** * Creates either NIO or regular depending on socketWriteTimeout. */ - protected Socket newSocket() throws IOException { + public Socket newSocket() throws IOException { return (dnConf.socketWriteTimeout > 0) ? SocketChannel.open().socket() : new Socket(); } @@ -2143,11 +2148,8 @@ public void run() { // // Header info // - Token accessToken = BlockTokenSecretManager.DUMMY_TOKEN; - if (isBlockTokenEnabled) { - accessToken = blockPoolTokenSecretManager.generateToken(b, - EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)); - } + Token accessToken = getBlockAccessToken(b, + EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)); long writeTimeout = dnConf.socketWriteTimeout + HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1); @@ -2214,6 +2216,19 @@ public void run() { } } + /*** + * Use BlockTokenSecretManager to generate block token for current user. + */ + public Token getBlockAccessToken(ExtendedBlock b, + EnumSet mode) throws IOException { + Token accessToken = + BlockTokenSecretManager.DUMMY_TOKEN; + if (isBlockTokenEnabled) { + accessToken = blockPoolTokenSecretManager.generateToken(b, mode); + } + return accessToken; + } + /** * Returns a new DataEncryptionKeyFactory that generates a key from the * BlockPoolTokenSecretManager, using the block pool ID of the given block. @@ -2221,7 +2236,7 @@ public void run() { * @param block for which the factory needs to create a key * @return DataEncryptionKeyFactory for block's block pool ID */ - DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock( + public DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock( final ExtendedBlock block) { return new DataEncryptionKeyFactory() { @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 6430308e7d5e0..c4e568f2841c0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -17,15 +17,68 @@ */ package org.apache.hadoop.hdfs.server.datanode.erasurecode; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; import java.util.Collection; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Future; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.BlockReader; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSPacket; +import org.apache.hadoop.hdfs.RemoteBlockReader2; +import org.apache.hadoop.hdfs.net.Peer; +import org.apache.hadoop.hdfs.net.TcpPeerServer; +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.datatransfer.BlockConstructionStage; +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; +import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; -import org.apache.hadoop.io.erasurecode.coder.AbstractErasureCoder; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripedReadResult; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.DataChecksum; + +import com.google.common.base.Preconditions; /** * ErasureCodingWorker handles the erasure coding recovery work commands. These @@ -34,41 +87,60 @@ * commands. */ public final class ErasureCodingWorker { - + private final Log LOG = DataNode.LOG; + + private final DataNode datanode; private Configuration conf; - RawErasureCoder rawEncoder = null; - RawErasureCoder rawDecoder = null; - public ErasureCodingWorker(Configuration conf) { + private ThreadPoolExecutor STRIPED_READ_TRHEAD_POOL; + private final int STRIPED_READ_THRESHOLD_MILLIS; + private final int STRIPED_READ_BUFFER_SIZE; + + public ErasureCodingWorker(Configuration conf, DataNode datanode) { + this.datanode = datanode; this.conf = conf; - initialize(); - } - /** - * Initializes the required resources for handling the erasure coding recovery - * work. - */ - public void initialize() { - // Right now directly used RS coder. Once other coders integration ready, we - // can load preferred codec here. - initializeErasureEncoder(); - initializeErasureDecoder(); + STRIPED_READ_THRESHOLD_MILLIS = conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT); + initializeStripedReadThreadPool(conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT)); + STRIPED_READ_BUFFER_SIZE = conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT); } - private void initializeErasureDecoder() { - rawDecoder = AbstractErasureCoder.createRawCoder(conf, - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, false); - if (rawDecoder == null) { - rawDecoder = new RSRawDecoder(); - } + private RawErasureEncoder newEncoder() { + return new RSRawEncoder(); + } + + private RawErasureDecoder newDecoder() { + return new RSRawDecoder(); } - private void initializeErasureEncoder() { - rawEncoder = AbstractErasureCoder.createRawCoder(conf, - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, true); - if (rawEncoder == null) { - rawEncoder = new RSRawEncoder(); - } + private void initializeStripedReadThreadPool(int num) { + STRIPED_READ_TRHEAD_POOL = new ThreadPoolExecutor(1, num, 60, + TimeUnit.SECONDS, new SynchronousQueue(), + new Daemon.DaemonFactory() { + private final AtomicInteger threadIndex = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable r) { + Thread t = super.newThread(r); + t.setName("stripedRead-" + threadIndex.getAndIncrement()); + return t; + } + }, new ThreadPoolExecutor.CallerRunsPolicy() { + @Override + public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) { + LOG.info("Execution for striped reading rejected, " + + "Executing in current thread"); + // will run in the current thread + super.rejectedExecution(runnable, e); + } + }); + STRIPED_READ_TRHEAD_POOL.allowCoreThreadTimeOut(true); } /** @@ -78,6 +150,765 @@ private void initializeErasureEncoder() { * BlockECRecoveryInfo */ public void processErasureCodingTasks(Collection ecTasks) { - // HDFS-7348 : Implement the actual recovery process + for (BlockECRecoveryInfo recoveryInfo : ecTasks) { + try { + new Daemon(new ReconstructAndTransferBlock(recoveryInfo)).start(); + } catch (Throwable e) { + LOG.warn("Failed to recover striped block " + + recoveryInfo.getExtendedBlock().getLocalBlock(), e); + } + } + } + + /** + * ReconstructAndTransferBlock recover one or more missed striped block in the + * striped block group, the minimum number of live striped blocks should be + * no less than data block number. + * + * | <- Striped Block Group -> | + * blk_0 blk_1 blk_2(*) blk_3 ... <- A striped block group + * | | | | + * v v v v + * +------+ +------+ +------+ +------+ + * |cell_0| |cell_1| |cell_2| |cell_3| ... + * +------+ +------+ +------+ +------+ + * |cell_4| |cell_5| |cell_6| |cell_7| ... + * +------+ +------+ +------+ +------+ + * |cell_8| |cell_9| |cell10| |cell11| ... + * +------+ +------+ +------+ +------+ + * ... ... ... ... + * + * + * We use following steps to recover striped block group, in each round, we + * recover bufferSize data until finish, the + * bufferSize is configurable and may be less or larger than + * cell size: + * step1: read bufferSize data from minimum number of sources + * required by recovery. + * step2: decode data for targets. + * step3: transfer data to targets. + * + * In step1, try to read bufferSize data from minimum number + * of sources , if there is corrupt or stale sources, read from new source + * will be scheduled. The best sources are remembered for next round and + * may be updated in each round. + * + * In step2, typically if source blocks we read are all data blocks, we + * need to call encode, and if there is one parity block, we need to call + * decode. Notice we only read once and recover all missed striped block + * if they are more than one. + * + * In step3, send the recovered data to targets by constructing packet + * and send them directly. Same as continuous block replication, we + * don't check the packet ack. Since the datanode doing the recovery work + * are one of the source datanodes, so the recovered data are sent + * remotely. + * + * There are some points we can do further improvements in next phase: + * 1. we can read the block file directly on the local datanode, + * currently we use remote block reader. (Notice short-circuit is not + * a good choice, see inline comments). + * 2. We need to check the packet ack for EC recovery? Since EC recovery + * is more expensive than continuous block replication, it needs to + * read from several other datanodes, should we make sure the + * recovered result received by targets? + */ + private class ReconstructAndTransferBlock implements Runnable { + private final int dataBlkNum; + private final int parityBlkNum; + private final int cellSize; + + private RawErasureEncoder encoder; + private RawErasureDecoder decoder; + + // Striped read buffer size + private int bufferSize; + + private final ExtendedBlock blockGroup; + // position in striped block + private long positionInBlock; + + // sources + private final short[] liveIndices; + private DatanodeInfo[] sources; + + private List stripedReaders; + + // targets + private DatanodeInfo[] targets; + private StorageType[] targetStorageTypes; + + private short[] targetIndices; + private ByteBuffer[] targetBuffers; + + private Socket[] targetSockets; + private DataOutputStream[] targetOutputStreams; + private DataInputStream[] targetInputStreams; + + private long[] blockOffset4Targets; + private long[] seqNo4Targets; + + private final int WRITE_PACKET_SIZE = 64 * 1024; + private DataChecksum checksum; + private int maxChunksPerPacket; + private byte[] packetBuf; + private byte[] checksumBuf; + private int bytesPerChecksum; + private int checksumSize; + + private CachingStrategy cachingStrategy; + + private Map, Integer> futures = new HashMap<>(); + private CompletionService readService = + new ExecutorCompletionService<>(STRIPED_READ_TRHEAD_POOL); + + ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) { + ECSchema schema = recoveryInfo.getECSchema(); + dataBlkNum = schema.getNumDataUnits(); + parityBlkNum = schema.getNumParityUnits(); + cellSize = schema.getChunkSize(); + + blockGroup = recoveryInfo.getExtendedBlock(); + + liveIndices = recoveryInfo.getLiveBlockIndices(); + sources = recoveryInfo.getSourceDnInfos(); + stripedReaders = new ArrayList<>(sources.length); + + Preconditions.checkArgument(liveIndices.length >= dataBlkNum, + "No enough live striped blocks."); + Preconditions.checkArgument(liveIndices.length == sources.length); + + targets = recoveryInfo.getTargetDnInfos(); + targetStorageTypes = recoveryInfo.getTargetStorageTypes(); + targetIndices = new short[targets.length]; + targetBuffers = new ByteBuffer[targets.length]; + + targetSockets = new Socket[targets.length]; + targetOutputStreams = new DataOutputStream[targets.length]; + targetInputStreams = new DataInputStream[targets.length]; + + blockOffset4Targets = new long[targets.length]; + seqNo4Targets = new long[targets.length]; + + for (int i = 0; i < targets.length; i++) { + blockOffset4Targets[i] = 0; + seqNo4Targets[i] = 0; + } + + getTargetIndices(); + cachingStrategy = CachingStrategy.newDefaultStrategy(); + } + + private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) { + return StripedBlockUtil.constructStripedBlock(blockGroup, cellSize, + dataBlkNum, i); + } + + private long getBlockLen(ExtendedBlock blockGroup, int i) { + return StripedBlockUtil.getStripedBlockLength(blockGroup.getNumBytes(), + cellSize, dataBlkNum, i); + } + + @Override + public void run() { + try { + // Store the indices of successfully read source + // This will be updated after doing real read. + int[] success = new int[dataBlkNum]; + + int nsuccess = 0; + for (int i = 0; i < sources.length && nsuccess < dataBlkNum; i++) { + StripedReader reader = new StripedReader(liveIndices[i]); + stripedReaders.add(reader); + + BlockReader blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[i]), 0, sources[i]); + if (blockReader != null) { + initChecksumAndBufferSizeIfNeeded(blockReader); + reader.blockReader = blockReader; + reader.buffer = ByteBuffer.allocate(bufferSize); + success[nsuccess++] = i; + } + } + + if (nsuccess < dataBlkNum) { + String error = "Can't find minimum sources required by " + + "recovery, block id: " + blockGroup.getBlockId(); + LOG.warn(error); + throw new IOException(error); + } + + for (int i = 0; i < targets.length; i++) { + targetBuffers[i] = ByteBuffer.allocate(bufferSize); + } + + checksumSize = checksum.getChecksumSize(); + int chunkSize = bytesPerChecksum + checksumSize; + maxChunksPerPacket = Math.max( + (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1); + int maxPacketSize = chunkSize * maxChunksPerPacket + + PacketHeader.PKT_MAX_HEADER_LEN; + + packetBuf = new byte[maxPacketSize]; + checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)]; + + // Store whether the target is success + boolean[] targetsStatus = new boolean[targets.length]; + if (initTargetStreams(targetsStatus) == 0) { + String error = "All targets are failed."; + LOG.warn(error); + throw new IOException(error); + } + + long firstStripedBlockLength = getBlockLen(blockGroup, 0); + while (positionInBlock < firstStripedBlockLength) { + int toRead = Math.min( + bufferSize, (int)(firstStripedBlockLength - positionInBlock)); + // step1: read minimum striped buffer size data required by recovery. + nsuccess = readMinimumStripedData4Recovery(success); + + if (nsuccess < dataBlkNum) { + String error = "Can't read data from minimum number of sources " + + "required by recovery, block id: " + blockGroup.getBlockId(); + LOG.warn(error); + throw new IOException(error); + } + + // step2: encode/decode to recover targets + long remaining = firstStripedBlockLength - positionInBlock; + int toRecoverLen = remaining < bufferSize ? + (int)remaining : bufferSize; + recoverTargets(success, targetsStatus, toRecoverLen); + + // step3: transfer data + if (transferData2Targets(targetsStatus) == 0) { + String error = "Transfer failed for all targets."; + LOG.warn(error); + throw new IOException(error); + } + + clearBuffers(); + positionInBlock += toRead; + } + + endTargetBlocks(targetsStatus); + + // Currently we don't check the acks for packets, this is similar as + // block replication. + } catch (Throwable e) { + LOG.warn("Failed to recover striped block: " + blockGroup); + } finally { + // close block readers + for (StripedReader stripedReader : stripedReaders) { + closeBlockReader(stripedReader.blockReader); + } + for (int i = 0; i < targets.length; i++) { + IOUtils.closeStream(targetOutputStreams[i]); + IOUtils.closeStream(targetInputStreams[i]); + IOUtils.closeStream(targetSockets[i]); + } + } + } + + // init checksum from block reader + private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) { + if (checksum == null) { + checksum = blockReader.getDataChecksum(); + bytesPerChecksum = checksum.getBytesPerChecksum(); + // The bufferSize is flat to divide bytesPerChecksum + int readBufferSize = STRIPED_READ_BUFFER_SIZE; + bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum : + readBufferSize - readBufferSize % bytesPerChecksum; + } else { + assert blockReader.getDataChecksum().equals(checksum); + } + } + + // assume liveIndices is not ordered. + private void getTargetIndices() { + BitSet bitset = new BitSet(dataBlkNum + parityBlkNum); + for (int i = 0; i < sources.length; i++) { + bitset.set(liveIndices[i]); + } + int m = 0; + for (int i = 0; i < dataBlkNum + parityBlkNum && m < targets.length; i++) { + if (!bitset.get(i)) { + targetIndices[m++] = (short)i; + } + } + } + + /** + * Read minimum striped buffer size data required by recovery. + * success list will be updated after read. + * + * Initially we only read from dataBlkNum sources, + * if timeout or failure for some source, we will try to schedule + * read from a new source. + */ + private int readMinimumStripedData4Recovery(int[] success) { + + BitSet used = new BitSet(sources.length); + for (int i = 0; i < dataBlkNum; i++) { + StripedReader reader = stripedReaders.get(success[i]); + Callable readCallable = readFromBlock( + reader.blockReader, reader.buffer); + Future f = readService.submit(readCallable); + futures.put(f, success[i]); + used.set(success[i]); + } + + int nsuccess = 0; + while (!futures.isEmpty()) { + try { + StripedReadResult result = + StripedBlockUtil.getNextCompletedStripedRead( + readService, futures, STRIPED_READ_THRESHOLD_MILLIS); + if (result.state == StripedReadResult.SUCCESSFUL) { + success[nsuccess++] = result.index; + if (nsuccess >= dataBlkNum) { + // cancel remaining reads if we read successfully from minimum + // number of sources required for recovery. + cancelReads(futures.keySet()); + futures.clear(); + break; + } + } else if (result.state == StripedReadResult.FAILED) { + // If read failed for some source, we should not use it anymore + // and schedule read from a new source. + StripedReader failedReader = stripedReaders.get(result.index); + closeBlockReader(failedReader.blockReader); + failedReader.blockReader = null; + scheduleNewRead(used); + } else if (result.state == StripedReadResult.TIMEOUT) { + // If timeout, we also schedule a new read. + scheduleNewRead(used); + } + } catch (InterruptedException e) { + LOG.info("Read data interrupted.", e); + break; + } + } + + return nsuccess; + } + + /** + * Return true if need to do encoding to recovery missed striped block. + */ + private boolean shouldEncode(int[] success) { + for (int i = 0; i < success.length; i++) { + if (stripedReaders.get(success[i]).index >= dataBlkNum) { + return false; + } + } + return true; + } + + private void paddingBufferToLen(ByteBuffer buffer, int len) { + int toPadding = len - buffer.position(); + for (int i = 0; i < toPadding; i++) { + buffer.put((byte) 0); + } + } + + // Initialize encoder + private void initEncoderIfNecessary() { + if (encoder == null) { + encoder = newEncoder(); + encoder.initialize(dataBlkNum, parityBlkNum, bufferSize); + } + } + + // Initialize decoder + private void initDecoderIfNecessary() { + if (decoder == null) { + decoder = newDecoder(); + decoder.initialize(dataBlkNum, parityBlkNum, bufferSize); + } + } + + private void recoverTargets(int[] success, boolean[] targetsStatus, + int toRecoverLen) { + if (shouldEncode(success)) { + initEncoderIfNecessary(); + ByteBuffer[] dataBuffers = new ByteBuffer[dataBlkNum]; + ByteBuffer[] parityBuffers = new ByteBuffer[parityBlkNum]; + for (int i = 0; i < dataBlkNum; i++) { + StripedReader reader = stripedReaders.get(i); + ByteBuffer buffer = reader.buffer; + paddingBufferToLen(buffer, toRecoverLen); + dataBuffers[i] = (ByteBuffer)buffer.flip(); + } + for (int i = dataBlkNum; i < stripedReaders.size(); i++) { + StripedReader reader = stripedReaders.get(i); + parityBuffers[reader.index - dataBlkNum] = cleanBuffer(reader.buffer); + } + for (int i = 0; i < targets.length; i++) { + parityBuffers[targetIndices[i] - dataBlkNum] = targetBuffers[i]; + } + for (int i = 0; i < parityBlkNum; i++) { + if (parityBuffers[i] == null) { + parityBuffers[i] = ByteBuffer.allocate(toRecoverLen); + } else { + parityBuffers[i].limit(toRecoverLen); + } + } + encoder.encode(dataBuffers, parityBuffers); + } else { + /////////// TODO: wait for HADOOP-11847 ///////////// + ////////// The current decode method always try to decode parityBlkNum number of data blocks. //////////// + initDecoderIfNecessary(); + ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum]; + for (int i = 0; i < success.length; i++) { + StripedReader reader = stripedReaders.get(success[i]); + ByteBuffer buffer = reader.buffer; + paddingBufferToLen(buffer, toRecoverLen); + int index = reader.index < dataBlkNum ? + reader.index + parityBlkNum : reader.index - dataBlkNum; + inputs[index] = (ByteBuffer)buffer.flip(); + } + int[] indices4Decode = new int[parityBlkNum]; + int m = 0; + for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { + if (inputs[i] == null) { + inputs[i] = ByteBuffer.allocate(toRecoverLen); + indices4Decode[m++] = i; + } + } + ByteBuffer[] outputs = new ByteBuffer[parityBlkNum]; + m = 0; + // targetIndices is subset of indices4Decode + for (int i = 0; i < parityBlkNum; i++) { + if (m < targetIndices.length && + (indices4Decode[i] - parityBlkNum) == targetIndices[m]) { + outputs[i] = targetBuffers[m++]; + outputs[i].limit(toRecoverLen); + } else { + outputs[i] = ByteBuffer.allocate(toRecoverLen); + } + } + + decoder.decode(inputs, indices4Decode, outputs); + + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + long blockLen = getBlockLen(blockGroup, targetIndices[i]); + long remaining = blockLen - positionInBlock; + if (remaining < 0) { + targetBuffers[i].limit(0); + } else if (remaining < toRecoverLen) { + targetBuffers[i].limit((int)remaining); + } + } + } + } + } + + /** + * Schedule read from a new source, we first try un-initial source, + * then try un-used source in this round and bypass failed source. + */ + private void scheduleNewRead(BitSet used) { + StripedReader reader = null; + int m = stripedReaders.size(); + while (m < sources.length && reader == null) { + reader = new StripedReader(liveIndices[m]); + BlockReader blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[m]), positionInBlock, sources[m]); + stripedReaders.add(reader); + if (blockReader != null) { + assert blockReader.getDataChecksum().equals(checksum); + reader.blockReader = blockReader; + reader.buffer = ByteBuffer.allocate(bufferSize); + } else { + m++; + reader = null; + } + } + + for (int i = 0; reader == null && i < stripedReaders.size(); i++) { + StripedReader r = stripedReaders.get(i); + if (r.blockReader != null && !used.get(i)) { + closeBlockReader(r.blockReader); + r.blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[i]), positionInBlock, + sources[i]); + if (r.blockReader != null) { + m = i; + reader = r; + } + } + } + + if (reader != null) { + Callable readCallable = readFromBlock( + reader.blockReader, reader.buffer); + Future f = readService.submit(readCallable); + futures.put(f, m); + used.set(m); + } + } + + // cancel all reads. + private void cancelReads(Collection> futures) { + for (Future future : futures) { + future.cancel(true); + } + } + + private Callable readFromBlock(final BlockReader reader, + final ByteBuffer buf) { + return new Callable() { + + @Override + public Void call() throws Exception { + try { + actualReadFromBlock(reader, buf); + return null; + } catch (IOException e) { + LOG.info(e.getMessage()); + throw e; + } + } + + }; + } + + /** + * Read bytes from block + */ + private void actualReadFromBlock(BlockReader reader, ByteBuffer buf) + throws IOException { + int len = buf.remaining(); + int n = 0; + while (n < len) { + int nread = reader.read(buf); + if (nread <= 0) { + break; + } + n += nread; + } + } + + // close block reader + private void closeBlockReader(BlockReader blockReader) { + try { + if (blockReader != null) { + blockReader.close(); + } + } catch (IOException e) { + // ignore + } + } + + private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) { + return NetUtils.createSocketAddr(dnInfo.getXferAddr( + datanode.getDnConf().getConnectToDnViaHostname())); + } + + private BlockReader newBlockReader(final ExtendedBlock block, + long startOffset, DatanodeInfo dnInfo) { + try { + InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo); + Token blockToken = datanode.getBlockAccessToken( + block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ)); + /* + * This can be further improved if the replica is local, then we can + * read directly from DN and need to check the replica is FINALIZED + * state, notice we should not use short-circuit local read which + * requires config for domain-socket in UNIX or legacy config in Windows. + */ + return RemoteBlockReader2.newBlockReader( + "dummy", block, blockToken, startOffset, block.getNumBytes(), true, + "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo, + null, cachingStrategy); + } catch (IOException e) { + return null; + } + } + + private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr, + Token blockToken, DatanodeID datanodeId) + throws IOException { + Peer peer = null; + boolean success = false; + Socket sock = null; + final int socketTimeout = datanode.getDnConf().getSocketTimeout(); + try { + sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); + NetUtils.connect(sock, addr, socketTimeout); + peer = TcpPeerServer.peerFromSocketAndKey(datanode.getSaslClient(), + sock, datanode.getDataEncryptionKeyFactoryForBlock(b), + blockToken, datanodeId); + peer.setReadTimeout(socketTimeout); + success = true; + return peer; + } finally { + if (!success) { + IOUtils.cleanup(LOG, peer); + IOUtils.closeSocket(sock); + } + } + } + + /** + * Send data to targets + */ + private int transferData2Targets(boolean[] targetsStatus) { + int nsuccess = 0; + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + boolean success = false; + try { + ByteBuffer buffer = targetBuffers[i]; + + if (buffer.remaining() == 0) { + continue; + } + + checksum.calculateChunkedSums( + buffer.array(), 0, buffer.remaining(), checksumBuf, 0); + + int ckOff = 0; + while (buffer.remaining() > 0) { + DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket, + blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false); + int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum; + int toWrite = buffer.remaining() > maxBytesToPacket ? + maxBytesToPacket : buffer.remaining(); + int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize; + packet.writeChecksum(checksumBuf, ckOff, ckLen); + ckOff += ckLen; + packet.writeData(buffer, toWrite); + + // Send packet + packet.writeTo(targetOutputStreams[i]); + + blockOffset4Targets[i] += toWrite; + nsuccess++; + success = true; + } + } catch (IOException e) { + LOG.warn(e.getMessage()); + } + targetsStatus[i] = success; + } + } + return nsuccess; + } + + /** + * clear all buffers + */ + private void clearBuffers() { + for (StripedReader stripedReader : stripedReaders) { + if (stripedReader.buffer != null) { + stripedReader.buffer.clear(); + } + } + + for (int i = 0; i < targetBuffers.length; i++) { + if (targetBuffers[i] != null) { + cleanBuffer(targetBuffers[i]); + } + } + } + + private ByteBuffer cleanBuffer(ByteBuffer buffer) { + Arrays.fill(buffer.array(), (byte) 0); + return (ByteBuffer)buffer.clear(); + } + + // send an empty packet to mark the end of the block + private void endTargetBlocks(boolean[] targetsStatus) { + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + try { + DFSPacket packet = new DFSPacket(packetBuf, 0, + blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true); + packet.writeTo(targetOutputStreams[i]); + targetOutputStreams[i].flush(); + } catch (IOException e) { + LOG.warn(e.getMessage()); + } + } + } + } + + /** + * Initialize output/input streams for transferring data to target + * and send create block request. + */ + private int initTargetStreams(boolean[] targetsStatus) { + int nsuccess = 0; + for (int i = 0; i < targets.length; i++) { + Socket socket = null; + DataOutputStream out = null; + DataInputStream in = null; + boolean success = false; + try { + InetSocketAddress targetAddr = + getSocketAddress4Transfer(targets[i]); + socket = datanode.newSocket(); + NetUtils.connect(socket, targetAddr, + datanode.getDnConf().getSocketTimeout()); + socket.setSoTimeout(datanode.getDnConf().getSocketTimeout()); + + ExtendedBlock block = getBlock(blockGroup, targetIndices[i]); + Token blockToken = + datanode.getBlockAccessToken(block, + EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)); + + long writeTimeout = datanode.getDnConf().getSocketWriteTimeout(); + OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout); + InputStream unbufIn = NetUtils.getInputStream(socket); + DataEncryptionKeyFactory keyFactory = + datanode.getDataEncryptionKeyFactoryForBlock(block); + IOStreamPair saslStreams = datanode.getSaslClient().socketSend( + socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]); + + unbufOut = saslStreams.out; + unbufIn = saslStreams.in; + + out = new DataOutputStream(new BufferedOutputStream(unbufOut, + HdfsServerConstants.SMALL_BUFFER_SIZE)); + in = new DataInputStream(unbufIn); + + DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId()); + new Sender(out).writeBlock(block, targetStorageTypes[i], + blockToken, "", new DatanodeInfo[]{targets[i]}, + new StorageType[]{targetStorageTypes[i]}, source, + BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, + checksum, cachingStrategy, false, false, null); + + targetSockets[i] = socket; + targetOutputStreams[i] = out; + targetInputStreams[i] = in; + nsuccess++; + success = true; + } catch (Throwable e) { + LOG.warn(e.getMessage()); + } finally { + if (!success) { + IOUtils.closeStream(out); + IOUtils.closeStream(in); + IOUtils.closeStream(socket); + } + } + targetsStatus[i] = success; + } + return nsuccess; + } + } + + private class StripedReader { + short index; + BlockReader blockReader; + ByteBuffer buffer; + + public StripedReader(short index) { + this.index = index; + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 24d4bfba844ef..45bbf6bf6d4c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.util; import com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -77,10 +78,8 @@ public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, int idxInReturnedLocs, int cellSize, int dataBlkNum, int idxInBlockGroup) { - final ExtendedBlock blk = new ExtendedBlock(bg.getBlock()); - blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup); - blk.setNumBytes(getInternalBlockLength(bg.getBlockSize(), - cellSize, dataBlkNum, idxInBlockGroup)); + final ExtendedBlock blk = constructInternalBlock( + bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); return new LocatedBlock(blk, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, @@ -90,6 +89,44 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, null); } + /** + * This method creates an internal {@link ExtendedBlock} at the given index + * of a block group. + */ + public static ExtendedBlock constructInternalBlock(ExtendedBlock blockGroup, + int cellSize, int dataBlkNum, int idxInBlockGroup) { + ExtendedBlock block = new ExtendedBlock(blockGroup); + block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup); + block.setNumBytes(getInternalBlockLength(blockGroup.getNumBytes(), + cellSize, dataBlkNum, idxInBlockGroup)); + return block; + } + + /** + * This method creates an internal {@link ExtendedBlock} at the given index + * of a block group, for both data and parity block. + */ + public static ExtendedBlock constructStripedBlock(ExtendedBlock blockGroup, + int cellSize, int dataBlkNum, int idxInBlockGroup) { + ExtendedBlock block = new ExtendedBlock(blockGroup); + block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup); + block.setNumBytes(getStripedBlockLength(blockGroup.getNumBytes(), cellSize, + dataBlkNum, idxInBlockGroup)); + return block; + } + + /** + * Returns an internal block length at the given index of a block group, + * for both data and parity block. + */ + public static long getStripedBlockLength(long numBytes, int cellSize, + int dataBlkNum, int idxInBlockGroup) { + // parity block length is the same as the first striped block length. + return StripedBlockUtil.getInternalBlockLength( + numBytes, cellSize, dataBlkNum, + idxInBlockGroup < dataBlkNum ? idxInBlockGroup : 0); + } + /** * Get the size of an internal block at the given index of a block group * @@ -208,8 +245,8 @@ public static ReadPortion[] planReadPortions(final int dataBlkNum, * @throws InterruptedException */ public static StripedReadResult getNextCompletedStripedRead( - CompletionService readService, Map, - Integer> futures, final long threshold) throws InterruptedException { + CompletionService readService, Map, Integer> futures, + final long threshold) throws InterruptedException { Preconditions.checkArgument(!futures.isEmpty()); Preconditions.checkArgument(threshold > 0); Future future = null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 7f0730b07f21c..f80212844bf32 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -2312,11 +2312,11 @@ - - dfs.datanode.block-pinning.enabled - false - Whether pin blocks on favored DataNode. - + + dfs.datanode.block-pinning.enabled + false + Whether pin blocks on favored DataNode. + dfs.client.block.write.locateFollowingBlock.initial.delay.ms @@ -2354,4 +2354,25 @@ + + dfs.datanode.stripedread.threshold.millis + 5000 + datanode striped read threshold in millisecond. + + + + + dfs.datanode.stripedread.threads + 20 + datanode striped read thread pool size. + + + + + dfs.datanode.stripedread.buffer.size + 262144 + datanode striped read buffer size. + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java new file mode 100644 index 0000000000000..b4f05d46b9a89 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -0,0 +1,356 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +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.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.BitSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TestRecoverStripedFile { + public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class); + + private static final int dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS; + private static final int parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS; + private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final int blockSize = cellSize * 3; + private static final int groupSize = dataBlkNum + parityBlkNum; + private static final int dnNum = groupSize + parityBlkNum; + + private MiniDFSCluster cluster; + private Configuration conf; + private DistributedFileSystem fs; + // Map: DatanodeID -> datanode index in cluster + private Map dnMap = new HashMap(); + + @Before + public void setup() throws IOException { + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, cellSize - 1); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();; + cluster.waitActive(); + + fs = cluster.getFileSystem(); + fs.getClient().createErasureCodingZone("/", null); + + List datanodes = cluster.getDataNodes(); + for (int i = 0; i < dnNum; i++) { + dnMap.put(datanodes.get(i).getDatanodeId(), i); + } + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverOneParityBlock", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverThreeParityBlocks() throws Exception { + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen, 0, 3); + } + + @Test(timeout = 120000) + public void testRecoverThreeDataBlocks() throws Exception { + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen, 1, 3); + } + + @Test(timeout = 120000) + public void testRecoverOneDataBlock() throws Exception { + ////TODO: TODO: wait for HADOOP-11847 + //int fileLen = 10 * blockSize + blockSize/10; + //assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1); + } + + @Test(timeout = 120000) + public void testRecoverAnyBlocks() throws Exception { + ////TODO: TODO: wait for HADOOP-11847 + //int fileLen = 3 * blockSize + blockSize/10; + //assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2); + } + + /** + * Test the file blocks recovery. + * 1. Check the replica is recovered in the target datanode, + * and verify the block replica length, generationStamp and content. + * 2. Read the file and verify content. + */ + private void assertFileBlocksRecovery(String fileName, int fileLen, + int recovery, int toRecoverBlockNum) throws Exception { + if (recovery != 0 && recovery != 1 && recovery != 2) { + Assert.fail("Invalid recovery: 0 is to recovery parity blocks," + + "1 is to recovery data blocks, 2 is any."); + } + if (toRecoverBlockNum < 1 || toRecoverBlockNum > parityBlkNum) { + Assert.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum); + } + + Path file = new Path(fileName); + + testCreateStripedFile(file, fileLen); + + LocatedBlocks locatedBlocks = getLocatedBlocks(file); + assertEquals(locatedBlocks.getFileLength(), fileLen); + + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + + DatanodeInfo[] storageInfos = lastBlock.getLocations(); + int[] indices = lastBlock.getBlockIndices(); + + BitSet bitset = new BitSet(dnNum); + for (DatanodeInfo storageInfo : storageInfos) { + bitset.set(dnMap.get(storageInfo)); + } + + int[] toDead = new int[toRecoverBlockNum]; + int n = 0; + for (int i = 0; i < indices.length; i++) { + if (n < toRecoverBlockNum) { + if (recovery == 0) { + if (indices[i] >= dataBlkNum) { + toDead[n++] = i; + } + } else if (recovery == 1) { + if (indices[i] < dataBlkNum) { + toDead[n++] = i; + } + } else { + toDead[n++] = i; + } + } else { + break; + } + } + + DatanodeInfo[] dataDNs = new DatanodeInfo[toRecoverBlockNum]; + int[] deadDnIndices = new int[toRecoverBlockNum]; + ExtendedBlock[] blocks = new ExtendedBlock[toRecoverBlockNum]; + File[] replicas = new File[toRecoverBlockNum]; + File[] metadatas = new File[toRecoverBlockNum]; + byte[][] replicaContents = new byte[toRecoverBlockNum][]; + for (int i = 0; i < toRecoverBlockNum; i++) { + dataDNs[i] = storageInfos[toDead[i]]; + deadDnIndices[i] = dnMap.get(dataDNs[i]); + + // Check the block replica file on deadDn before it dead. + blocks[i] = StripedBlockUtil.constructStripedBlock( + lastBlock.getBlock(), cellSize, dataBlkNum, indices[toDead[i]]); + replicas[i] = cluster.getBlockFile(deadDnIndices[i], blocks[i]); + metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]); + // the block replica on the datanode should be the same as expected + assertEquals(replicas[i].length(), + StripedBlockUtil.getStripedBlockLength( + lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]])); + assertTrue(metadatas[i].getName(). + endsWith(blocks[i].getGenerationStamp() + ".meta")); + replicaContents[i] = readReplica(replicas[i]); + } + + try { + DatanodeID[] dnIDs = new DatanodeID[toRecoverBlockNum]; + for (int i = 0; i < toRecoverBlockNum; i++) { + /* + * Kill the datanode which contains one replica + * We need to make sure it dead in namenode: clear its update time and + * trigger NN to check heartbeat. + */ + DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]); + dn.shutdown(); + dnIDs[i] = dn.getDatanodeId(); + } + setDataNodesDead(dnIDs); + + + // Check the locatedBlocks of the file again + locatedBlocks = getLocatedBlocks(file); + lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + storageInfos = lastBlock.getLocations(); + assertEquals(storageInfos.length, groupSize - toRecoverBlockNum); + + int[] targetDNs = new int[dnNum - groupSize]; + n = 0; + for (int i = 0; i < dnNum; i++) { + if (!bitset.get(i)) { // not contain replica of the block. + targetDNs[n++] = i; + } + } + + waitForRecoveryFinished(file); + + targetDNs = sortTargetsByReplicas(blocks, targetDNs); + + // Check the replica on the new target node. + for (int i = 0; i < toRecoverBlockNum; i++) { + File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]); + File metadataAfterRecovery = + cluster.getBlockMetadataFile(targetDNs[i], blocks[i]); + assertEquals(replicaAfterRecovery.length(), replicas[i].length()); + assertTrue(metadataAfterRecovery.getName(). + endsWith(blocks[i].getGenerationStamp() + ".meta")); + byte[] replicaContentAfterRecovery = readReplica(replicaAfterRecovery); + + Assert.assertArrayEquals(replicaContents[i], replicaContentAfterRecovery); + } + } finally { + for (int i = 0; i < toRecoverBlockNum; i++) { + restartDataNode(toDead[i]); + } + cluster.waitActive(); + } + fs.delete(file, true); + } + + private void setDataNodesDead(DatanodeID[] dnIDs) throws IOException { + for (DatanodeID dn : dnIDs) { + DatanodeDescriptor dnd = + NameNodeAdapter.getDatanode(cluster.getNamesystem(), dn); + DFSTestUtil.setDatanodeDead(dnd); + } + + BlockManagerTestUtil.checkHeartbeat(cluster.getNamesystem().getBlockManager()); + } + + private void restartDataNode(int dn) { + try { + cluster.restartDataNode(dn, true, true); + } catch (IOException e) { + } + } + + private int[] sortTargetsByReplicas(ExtendedBlock[] blocks, int[] targetDNs) { + int[] result = new int[blocks.length]; + for (int i = 0; i < blocks.length; i++) { + result[i] = -1; + for (int j = 0; j < targetDNs.length; j++) { + if (targetDNs[j] != -1) { + File replica = cluster.getBlockFile(targetDNs[j], blocks[i]); + if (replica != null) { + result[i] = targetDNs[j]; + targetDNs[j] = -1; + break; + } + } + } + if (result[i] == -1) { + Assert.fail("Failed to recover striped block: " + blocks[i].getBlockId()); + } + } + return result; + } + + private byte[] readReplica(File replica) throws IOException { + int length = (int)replica.length(); + ByteArrayOutputStream content = new ByteArrayOutputStream(length); + FileInputStream in = new FileInputStream(replica); + try { + byte[] buffer = new byte[1024]; + int total = 0; + while (total < length) { + int n = in.read(buffer); + if (n <= 0) { + break; + } + content.write(buffer, 0, n); + total += n; + } + if (total < length) { + Assert.fail("Failed to read all content of replica"); + } + return content.toByteArray(); + } finally { + in.close(); + } + } + + private LocatedBlocks waitForRecoveryFinished(Path file) throws Exception { + final int ATTEMPTS = 60; + for (int i = 0; i < ATTEMPTS; i++) { + LocatedBlocks locatedBlocks = getLocatedBlocks(file); + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + DatanodeInfo[] storageInfos = lastBlock.getLocations(); + if (storageInfos.length >= groupSize) { + return locatedBlocks; + } + Thread.sleep(1000); + } + throw new IOException ("Time out waiting for EC block recovery."); + } + + private LocatedBlocks getLocatedBlocks(Path file) throws IOException { + return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE); + } + + private void testCreateStripedFile(Path file, int dataLen) + throws IOException { + final byte[] data = new byte[dataLen]; + DFSUtil.getRandom().nextBytes(data); + writeContents(file, data); + } + + void writeContents(Path file, byte[] contents) + throws IOException { + FSDataOutputStream out = fs.create(file); + try { + out.write(contents, 0, contents.length); + } finally { + out.close(); + } + } +} From cea46f79b0fabdf42bd3ac6e25fa263b75791f57 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 6 May 2015 15:34:37 -0700 Subject: [PATCH 101/212] HDFS-8334. Erasure coding: rename DFSStripedInputStream related test classes. Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 + .../hdfs/TestDFSStripedInputStream.java | 365 ++++++++---------- .../hadoop/hdfs/TestReadStripedFile.java | 218 ----------- .../hadoop/hdfs/TestWriteReadStripedFile.java | 261 +++++++++++++ 4 files changed, 427 insertions(+), 422 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0d2d4485e1647..8729f8ae48ee3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -178,3 +178,8 @@ HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. (Yi Liu via Zhe Zhang) + + HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng) + + HDFS-8334. Erasure coding: rename DFSStripedInputStream related test + classes. (Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 11cdf7b498563..a1f704d5b1c28 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -17,245 +17,202 @@ */ package org.apache.hadoop.hdfs; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ECInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; - -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; public class TestDFSStripedInputStream { - private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - - - private static DistributedFileSystem fs; - private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - private final static int stripesPerBlock = 4; - static int blockSize = cellSize * stripesPerBlock; - static int numDNs = dataBlocks + parityBlocks + 2; - - private static MiniDFSCluster cluster; - @BeforeClass - public static void setup() throws IOException { - Configuration conf = new Configuration(); - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + public static final Log LOG = LogFactory.getLog(TestDFSStripedInputStream.class); + + private MiniDFSCluster cluster; + private Configuration conf = new Configuration(); + private DistributedFileSystem fs; + private final Path dirPath = new Path("/striped"); + private Path filePath = new Path(dirPath, "file"); + private ECInfo info = new ECInfo(filePath.toString(), + ECSchemaManager.getSystemDefaultSchema()); + private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; + private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int NUM_STRIPE_PER_BLOCK = 2; + private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; + private final int BLOCK_GROUP_SIZE = DATA_BLK_NUM * INTERNAL_BLOCK_SIZE; + + @Before + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE); + SimulatedFSDataset.setFactory(conf); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes( + DATA_BLK_NUM + PARITY_BLK_NUM).build(); + cluster.waitActive(); fs = cluster.getFileSystem(); + fs.mkdirs(dirPath); + fs.getClient().createErasureCodingZone(dirPath.toString(), null); } - @AfterClass - public static void tearDown() { + @After + public void tearDown() { if (cluster != null) { cluster.shutdown(); } } + /** + * Test {@link DFSStripedInputStream#getBlockAt(long)} + */ @Test - public void testFileEmpty() throws IOException { - testOneFileUsingDFSStripedInputStream("/EmptyFile", 0); - } - - @Test - public void testFileSmallerThanOneCell1() throws IOException { - testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1); - } - - @Test - public void testFileSmallerThanOneCell2() throws IOException { - testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1); - } - - @Test - public void testFileEqualsWithOneCell() throws IOException { - testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize); - } - - @Test - public void testFileSmallerThanOneStripe1() throws IOException { - testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", - cellSize * dataBlocks - 1); - } - - @Test - public void testFileSmallerThanOneStripe2() throws IOException { - testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", - cellSize + 123); - } - - @Test - public void testFileEqualsWithOneStripe() throws IOException { - testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", - cellSize * dataBlocks); - } - - @Test - public void testFileMoreThanOneStripe1() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", - cellSize * dataBlocks + 123); - } - - @Test - public void testFileMoreThanOneStripe2() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", - cellSize * dataBlocks + cellSize * dataBlocks + 123); - } - - @Test - public void testLessThanFullBlockGroup() throws IOException { - testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup", - cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); - } - - @Test - public void testFileFullBlockGroup() throws IOException { - testOneFileUsingDFSStripedInputStream("/FullBlockGroup", - blockSize * dataBlocks); + public void testGetBlock() throws Exception { + final int numBlocks = 4; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); + final DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info); + + List lbList = lbs.getLocatedBlocks(); + for (LocatedBlock aLbList : lbList) { + LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList; + LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb, + CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); + for (int j = 0; j < DATA_BLK_NUM; j++) { + LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset()); + assertEquals(blks[j].getBlock(), refreshed.getBlock()); + assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset()); + assertArrayEquals(blks[j].getLocations(), refreshed.getLocations()); + } + } } @Test - public void testFileMoreThanABlockGroup1() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", - blockSize * dataBlocks + 123); - } + public void testPread() throws Exception { + final int numBlocks = 2; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCK_GROUP_SIZE); + + assert lbs.get(0) instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); + } + DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), + filePath.toString(), false, info); + int readSize = BLOCK_GROUP_SIZE; + byte[] readBuffer = new byte[readSize]; + int ret = in.read(0, readBuffer, 0, readSize); - @Test - public void testFileMoreThanABlockGroup2() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", - blockSize * dataBlocks + cellSize+ 123); + assertEquals(readSize, ret); + // TODO: verify read results with patterned data from HDFS-8117 } - @Test - public void testFileMoreThanABlockGroup3() throws IOException { - testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3", - blockSize * dataBlocks * 3 + cellSize * dataBlocks - + cellSize + 123); - } - - private byte[] generateBytes(int cnt) { - byte[] bytes = new byte[cnt]; - for (int i = 0; i < cnt; i++) { - bytes[i] = getByte(i); + public void testStatefulRead() throws Exception { + testStatefulRead(false, false); + testStatefulRead(true, false); + testStatefulRead(true, true); + } + + private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) + throws Exception { + final int numBlocks = 2; + final int fileSize = numBlocks * BLOCK_GROUP_SIZE; + if (cellMisalignPacket) { + conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1); + tearDown(); + setup(); } - return bytes; - } - - private byte getByte(long pos) { - final int mod = 29; - return (byte) (pos % mod + 1); - } - - private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) - throws IOException { - Path testPath = new Path(src); - final byte[] bytes = generateBytes(writeBytes); - DFSTestUtil.writeFile(fs, testPath, new String(bytes)); - - //check file length - FileStatus status = fs.getFileStatus(testPath); - long fileLength = status.getLen(); - Assert.assertEquals("File length should be the same", - writeBytes, fileLength); - - // pread - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - byte[] buf = new byte[writeBytes + 100]; - int readLen = fsdis.read(0, buf, 0, buf.length); - readLen = readLen >= 0 ? readLen : 0; - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), - buf[i]); + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, fileSize); + + assert lbs.getLocatedBlocks().size() == numBlocks; + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + assert lb instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lb); + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); } } - // stateful read with byte array - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - byte[] buf = new byte[writeBytes + 100]; - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf, readLen, buf.length - readLen); - if (ret > 0) { - readLen += ret; + DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), filePath.toString(), + false, info); + + byte[] expected = new byte[fileSize]; + + for (LocatedBlock bg : lbs.getLocatedBlocks()) { + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = (int) bg.getStartOffset() + + i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[posInFile] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } } - } while (ret >= 0); - readLen = readLen >= 0 ? readLen : 0; - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), - buf[i]); } } - // stateful read with ByteBuffer - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100); - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf); - if (ret > 0) { - readLen += ret; - } - } while (ret >= 0); - readLen = readLen >= 0 ? readLen : 0; - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), - buf.array()[i]); + if (useByteBuffer) { + ByteBuffer readBuffer = ByteBuffer.allocate(fileSize); + int done = 0; + while (done < fileSize) { + int ret = in.read(readBuffer); + assertTrue(ret > 0); + done += ret; } + assertArrayEquals(expected, readBuffer.array()); + } else { + byte[] readBuffer = new byte[fileSize]; + int done = 0; + while (done < fileSize) { + int ret = in.read(readBuffer, done, fileSize - done); + assertTrue(ret > 0); + done += ret; + } + assertArrayEquals(expected, readBuffer); } - - // stateful read with 1KB size byte array - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - final byte[] result = new byte[writeBytes]; - final byte[] buf = new byte[1024]; - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf, 0, buf.length); - if (ret > 0) { - System.arraycopy(buf, 0, result, readLen, ret); - readLen += ret; - } - } while (ret >= 0); - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - Assert.assertArrayEquals(bytes, result); - } - - // stateful read using ByteBuffer with 1KB size - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - final ByteBuffer result = ByteBuffer.allocate(writeBytes); - final ByteBuffer buf = ByteBuffer.allocate(1024); - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf); - if (ret > 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); - } - } while (ret >= 0); - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - Assert.assertArrayEquals(bytes, result.array()); - } + fs.delete(filePath, true); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java deleted file mode 100644 index 1ad480e7f2952..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java +++ /dev/null @@ -1,218 +0,0 @@ -/** - * 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.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.ECInfo; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; -import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.List; - -public class TestReadStripedFile { - - public static final Log LOG = LogFactory.getLog(TestReadStripedFile.class); - - private MiniDFSCluster cluster; - private Configuration conf = new Configuration(); - private DistributedFileSystem fs; - private final Path dirPath = new Path("/striped"); - private Path filePath = new Path(dirPath, "file"); - private ECInfo info = new ECInfo(filePath.toString(), - ECSchemaManager.getSystemDefaultSchema()); - private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; - private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; - private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - private final int NUM_STRIPE_PER_BLOCK = 2; - private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; - private final int BLOCK_GROUP_SIZE = DATA_BLK_NUM * INTERNAL_BLOCK_SIZE; - - @Before - public void setup() throws IOException { - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE); - SimulatedFSDataset.setFactory(conf); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes( - DATA_BLK_NUM + PARITY_BLK_NUM).build(); - cluster.waitActive(); - fs = cluster.getFileSystem(); - fs.mkdirs(dirPath); - fs.getClient().createErasureCodingZone(dirPath.toString(), null); - } - - @After - public void tearDown() { - if (cluster != null) { - cluster.shutdown(); - } - } - - /** - * Test {@link DFSStripedInputStream#getBlockAt(long)} - */ - @Test - public void testGetBlock() throws Exception { - final int numBlocks = 4; - DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, - NUM_STRIPE_PER_BLOCK, false); - LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); - final DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info); - - List lbList = lbs.getLocatedBlocks(); - for (LocatedBlock aLbList : lbList) { - LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList; - LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb, - CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); - for (int j = 0; j < DATA_BLK_NUM; j++) { - LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset()); - assertEquals(blks[j].getBlock(), refreshed.getBlock()); - assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset()); - assertArrayEquals(blks[j].getLocations(), refreshed.getLocations()); - } - } - } - - @Test - public void testPread() throws Exception { - final int numBlocks = 2; - DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, - NUM_STRIPE_PER_BLOCK, false); - LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, BLOCK_GROUP_SIZE); - - assert lbs.get(0) instanceof LocatedStripedBlock; - LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); - for (int i = 0; i < DATA_BLK_NUM; i++) { - Block blk = new Block(bg.getBlock().getBlockId() + i, - NUM_STRIPE_PER_BLOCK * CELLSIZE, - bg.getBlock().getGenerationStamp()); - blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); - cluster.injectBlocks(i, Arrays.asList(blk), - bg.getBlock().getBlockPoolId()); - } - DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, info); - int readSize = BLOCK_GROUP_SIZE; - byte[] readBuffer = new byte[readSize]; - int ret = in.read(0, readBuffer, 0, readSize); - - assertEquals(readSize, ret); - // TODO: verify read results with patterned data from HDFS-8117 - } - - @Test - public void testStatefulRead() throws Exception { - testStatefulRead(false, false); - testStatefulRead(true, false); - testStatefulRead(true, true); - } - - private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) - throws Exception { - final int numBlocks = 2; - final int fileSize = numBlocks * BLOCK_GROUP_SIZE; - if (cellMisalignPacket) { - conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1); - tearDown(); - setup(); - } - DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, - NUM_STRIPE_PER_BLOCK, false); - LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, fileSize); - - assert lbs.getLocatedBlocks().size() == numBlocks; - for (LocatedBlock lb : lbs.getLocatedBlocks()) { - assert lb instanceof LocatedStripedBlock; - LocatedStripedBlock bg = (LocatedStripedBlock)(lb); - for (int i = 0; i < DATA_BLK_NUM; i++) { - Block blk = new Block(bg.getBlock().getBlockId() + i, - NUM_STRIPE_PER_BLOCK * CELLSIZE, - bg.getBlock().getGenerationStamp()); - blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); - cluster.injectBlocks(i, Arrays.asList(blk), - bg.getBlock().getBlockPoolId()); - } - } - - DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), filePath.toString(), - false, info); - - byte[] expected = new byte[fileSize]; - - for (LocatedBlock bg : lbs.getLocatedBlocks()) { - /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ - for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { - for (int j = 0; j < DATA_BLK_NUM; j++) { - for (int k = 0; k < CELLSIZE; k++) { - int posInBlk = i * CELLSIZE + k; - int posInFile = (int) bg.getStartOffset() + - i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; - expected[posInFile] = SimulatedFSDataset.simulatedByte( - new Block(bg.getBlock().getBlockId() + j), posInBlk); - } - } - } - } - - if (useByteBuffer) { - ByteBuffer readBuffer = ByteBuffer.allocate(fileSize); - int done = 0; - while (done < fileSize) { - int ret = in.read(readBuffer); - assertTrue(ret > 0); - done += ret; - } - assertArrayEquals(expected, readBuffer.array()); - } else { - byte[] readBuffer = new byte[fileSize]; - int done = 0; - while (done < fileSize) { - int ret = in.read(readBuffer, done, fileSize - done); - assertTrue(ret > 0); - done += ret; - } - assertArrayEquals(expected, readBuffer); - } - fs.delete(filePath, true); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java new file mode 100644 index 0000000000000..eacc6edddf636 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -0,0 +1,261 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public class TestWriteReadStripedFile { + private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + + + private static DistributedFileSystem fs; + private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + static int blockSize = cellSize * stripesPerBlock; + static int numDNs = dataBlocks + parityBlocks + 2; + + private static MiniDFSCluster cluster; + + @BeforeClass + public static void setup() throws IOException { + Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + fs = cluster.getFileSystem(); + } + + @AfterClass + public static void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testFileEmpty() throws IOException { + testOneFileUsingDFSStripedInputStream("/EmptyFile", 0); + } + + @Test + public void testFileSmallerThanOneCell1() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1); + } + + @Test + public void testFileSmallerThanOneCell2() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1); + } + + @Test + public void testFileEqualsWithOneCell() throws IOException { + testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize); + } + + @Test + public void testFileSmallerThanOneStripe1() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", + cellSize * dataBlocks - 1); + } + + @Test + public void testFileSmallerThanOneStripe2() throws IOException { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", + cellSize + 123); + } + + @Test + public void testFileEqualsWithOneStripe() throws IOException { + testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", + cellSize * dataBlocks); + } + + @Test + public void testFileMoreThanOneStripe1() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", + cellSize * dataBlocks + 123); + } + + @Test + public void testFileMoreThanOneStripe2() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", + cellSize * dataBlocks + cellSize * dataBlocks + 123); + } + + @Test + public void testLessThanFullBlockGroup() throws IOException { + testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); + } + + @Test + public void testFileFullBlockGroup() throws IOException { + testOneFileUsingDFSStripedInputStream("/FullBlockGroup", + blockSize * dataBlocks); + } + + @Test + public void testFileMoreThanABlockGroup1() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", + blockSize * dataBlocks + 123); + } + + @Test + public void testFileMoreThanABlockGroup2() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", + blockSize * dataBlocks + cellSize+ 123); + } + + + @Test + public void testFileMoreThanABlockGroup3() throws IOException { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123); + } + + private byte[] generateBytes(int cnt) { + byte[] bytes = new byte[cnt]; + for (int i = 0; i < cnt; i++) { + bytes[i] = getByte(i); + } + return bytes; + } + + private byte getByte(long pos) { + final int mod = 29; + return (byte) (pos % mod + 1); + } + + private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) + throws IOException { + Path testPath = new Path(src); + final byte[] bytes = generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, testPath, new String(bytes)); + + //check file length + FileStatus status = fs.getFileStatus(testPath); + long fileLength = status.getLen(); + Assert.assertEquals("File length should be the same", + writeBytes, fileLength); + + // pread + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + byte[] buf = new byte[writeBytes + 100]; + int readLen = fsdis.read(0, buf, 0, buf.length); + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + for (int i = 0; i < writeBytes; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), + buf[i]); + } + } + + // stateful read with byte array + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + byte[] buf = new byte[writeBytes + 100]; + int readLen = 0; + int ret; + do { + ret = fsdis.read(buf, readLen, buf.length - readLen); + if (ret > 0) { + readLen += ret; + } + } while (ret >= 0); + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + for (int i = 0; i < writeBytes; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), + buf[i]); + } + } + + // stateful read with ByteBuffer + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100); + int readLen = 0; + int ret; + do { + ret = fsdis.read(buf); + if (ret > 0) { + readLen += ret; + } + } while (ret >= 0); + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + for (int i = 0; i < writeBytes; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), + buf.array()[i]); + } + } + + // stateful read with 1KB size byte array + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + final byte[] result = new byte[writeBytes]; + final byte[] buf = new byte[1024]; + int readLen = 0; + int ret; + do { + ret = fsdis.read(buf, 0, buf.length); + if (ret > 0) { + System.arraycopy(buf, 0, result, readLen, ret); + readLen += ret; + } + } while (ret >= 0); + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + Assert.assertArrayEquals(bytes, result); + } + + // stateful read using ByteBuffer with 1KB size + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + final ByteBuffer result = ByteBuffer.allocate(writeBytes); + final ByteBuffer buf = ByteBuffer.allocate(1024); + int readLen = 0; + int ret; + do { + ret = fsdis.read(buf); + if (ret > 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } while (ret >= 0); + Assert.assertEquals("The length of file should be the same to write size", + writeBytes, readLen); + Assert.assertArrayEquals(bytes, result.array()); + } + } +} From ac97edd1abcca2be93aa3a8dcdc642734c7c00ab Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Thu, 7 May 2015 16:26:01 +0530 Subject: [PATCH 102/212] HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding. Contributed by Uma Maheswara Rao G --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../org/apache/hadoop/hdfs/DFSClient.java | 10 ++++---- .../hadoop/hdfs/DFSStripedInputStream.java | 2 +- .../hadoop/hdfs/DistributedFileSystem.java | 10 ++++---- .../hadoop/hdfs/protocol/ClientProtocol.java | 4 ++-- .../{ECInfo.java => ErasureCodingInfo.java} | 4 ++-- ...neInfo.java => ErasureCodingZoneInfo.java} | 4 ++-- ...amenodeProtocolServerSideTranslatorPB.java | 18 +++++++------- .../ClientNamenodeProtocolTranslatorPB.java | 16 ++++++------- .../hadoop/hdfs/protocolPB/PBHelper.java | 24 +++++++++---------- ...r.java => ErasureCodingSchemaManager.java} | 4 ++-- .../namenode/ErasureCodingZoneManager.java | 12 +++++----- .../hdfs/server/namenode/FSDirectory.java | 4 ++-- .../hdfs/server/namenode/FSNamesystem.java | 24 +++++++++---------- .../server/namenode/NameNodeRpcServer.java | 8 +++---- .../hdfs/tools/erasurecode/ECCommand.java | 4 ++-- .../main/proto/ClientNamenodeProtocol.proto | 4 ++-- .../src/main/proto/erasurecoding.proto | 16 ++++++------- .../hdfs/TestDFSStripedInputStream.java | 8 +++---- .../org/apache/hadoop/hdfs/TestECSchemas.java | 2 +- .../hadoop/hdfs/TestErasureCodingZones.java | 10 ++++---- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 10 ++++---- .../server/namenode/TestStripedINodeFile.java | 16 ++++++------- 23 files changed, 110 insertions(+), 107 deletions(-) rename hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/{ECInfo.java => ErasureCodingInfo.java} (92%) rename hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/{ECZoneInfo.java => ErasureCodingZoneInfo.java} (93%) rename hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/{ECSchemaManager.java => ErasureCodingSchemaManager.java} (97%) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8729f8ae48ee3..11e83760226be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -183,3 +183,6 @@ HDFS-8334. Erasure coding: rename DFSStripedInputStream related test classes. (Zhe Zhang) + + HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding + (umamahesh) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 84e7871e53731..8f250fdd34249 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -119,8 +119,8 @@ 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.ECInfo; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -1193,7 +1193,7 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) // Get block info from namenode TraceScope scope = getPathTraceScope("newDFSInputStream", src); try { - ECInfo info = getErasureCodingInfo(src); + ErasureCodingInfo info = getErasureCodingInfo(src); if (info != null) { return new DFSStripedInputStream(this, src, verifyChecksum, info); } else { @@ -3134,7 +3134,7 @@ public void checkAccess(String src, FsAction mode) throws IOException { } } - public ECInfo getErasureCodingInfo(String src) throws IOException { + public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException { checkOpen(); TraceScope scope = getPathTraceScope("getErasureCodingInfo", src); try { @@ -3356,7 +3356,7 @@ TraceScope getSrcDstTraceScope(String description, String src, String dst) { * @return Returns the zone information if path is in EC Zone, null otherwise * @throws IOException */ - public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException { checkOpen(); TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 13c4743040c9d..7cb7b6dd8603d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -127,7 +127,7 @@ boolean include(long pos) { private final CompletionService readingService; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, - ECInfo ecInfo) throws IOException { + ErasureCodingInfo ecInfo) throws IOException { super(dfsClient, src, verifyChecksum); // ECInfo is restored from NN just before reading striped file. assert ecInfo != null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index d471271295db6..0cd8334cf2f5d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -75,7 +75,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -2315,18 +2315,18 @@ public Void next(final FileSystem fs, final Path p) throws IOException { * @return Returns the zone information if path is in EC zone, null otherwise * @throws IOException */ - public ECZoneInfo getErasureCodingZoneInfo(final Path path) + public ErasureCodingZoneInfo getErasureCodingZoneInfo(final Path path) throws IOException { Path absF = fixRelativePart(path); - return new FileSystemLinkResolver() { + return new FileSystemLinkResolver() { @Override - public ECZoneInfo doCall(final Path p) throws IOException, + public ErasureCodingZoneInfo doCall(final Path p) throws IOException, UnresolvedLinkException { return dfs.getErasureCodingZoneInfo(getPathName(p)); } @Override - public ECZoneInfo next(final FileSystem fs, final Path p) + public ErasureCodingZoneInfo next(final FileSystem fs, final Path p) throws IOException { if (fs instanceof DistributedFileSystem) { DistributedFileSystem myDfs = (DistributedFileSystem) fs; 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 76e2d12d9ba3c..b0b457cf415cc 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 @@ -1474,7 +1474,7 @@ public void createErasureCodingZone(String src, ECSchema schema) * @throws IOException */ @Idempotent - public ECInfo getErasureCodingInfo(String src) throws IOException; + public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException; /** * Gets list of ECSchemas loaded in Namenode @@ -1492,5 +1492,5 @@ public void createErasureCodingZone(String src, ECSchema schema) * @throws IOException */ @Idempotent - public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException; + public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java similarity index 92% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java index ca642c233777f..bad09b3d19012 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java @@ -22,11 +22,11 @@ /** * Class to provide information, such as ECSchema, for a file/block. */ -public class ECInfo { +public class ErasureCodingInfo { private final String src; private final ECSchema schema; - public ECInfo(String src, ECSchema schema) { + public ErasureCodingInfo(String src, ECSchema schema) { this.src = src; this.schema = schema; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java similarity index 93% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java index ecfb92e789b24..ec0efbd345466 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java @@ -21,12 +21,12 @@ /** * Information about the EC Zone at the specified path. */ -public class ECZoneInfo { +public class ErasureCodingZoneInfo { private String dir; private ECSchema schema; - public ECZoneInfo(String dir, ECSchema schema) { + public ErasureCodingZoneInfo(String dir, ECSchema schema) { this.dir = dir; this.schema = schema; } 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 f9bdb71fe6fbf..bae753b72bb88 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 @@ -35,8 +35,8 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ECInfo; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -202,8 +202,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; @@ -1527,7 +1527,7 @@ public GetEditsFromTxidResponseProto getEditsFromTxid(RpcController controller, public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller, GetErasureCodingInfoRequestProto request) throws ServiceException { try { - ECInfo ecInfo = server.getErasureCodingInfo(request.getSrc()); + ErasureCodingInfo ecInfo = server.getErasureCodingInfo(request.getSrc()); GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto .newBuilder(); if (ecInfo != null) { @@ -1556,11 +1556,11 @@ public GetECSchemasResponseProto getECSchemas(RpcController controller, } @Override - public GetECZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller, - GetECZoneInfoRequestProto request) throws ServiceException { + public GetErasureCodingZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller, + GetErasureCodingZoneInfoRequestProto request) throws ServiceException { try { - ECZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc()); - GetECZoneInfoResponseProto.Builder builder = GetECZoneInfoResponseProto.newBuilder(); + ErasureCodingZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc()); + GetErasureCodingZoneInfoResponseProto.Builder builder = GetErasureCodingZoneInfoResponseProto.newBuilder(); if (ecZoneInfo != null) { builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo)); } 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 6c1c9710defcc..014fcef80bf91 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 @@ -58,8 +58,8 @@ 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.ECInfo; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -168,8 +168,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; @@ -1550,7 +1550,7 @@ public EventBatchList getEditsFromTxid(long txid) throws IOException { } @Override - public ECInfo getErasureCodingInfo(String src) throws IOException { + public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException { GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder() .setSrc(src).build(); try { @@ -1581,11 +1581,11 @@ public ECSchema[] getECSchemas() throws IOException { } @Override - public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { - GetECZoneInfoRequestProto req = GetECZoneInfoRequestProto.newBuilder() + public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + GetErasureCodingZoneInfoRequestProto req = GetErasureCodingZoneInfoRequestProto.newBuilder() .setSrc(src).build(); try { - GetECZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo( + GetErasureCodingZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo( null, req); if (response.hasECZoneInfo()) { return PBHelper.convertECZoneInfo(response.getECZoneInfo()); 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 e230232767fc5..26bdf3480ff07 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 @@ -77,13 +77,13 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; -import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -135,10 +135,10 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto; 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.BlockStoragePolicyProto; @@ -3117,13 +3117,13 @@ public static BlockReportContextProto convert(BlockReportContext context) { build(); } - public static ECInfo convertECInfo(ECInfoProto ecInfoProto) { - return new ECInfo(ecInfoProto.getSrc(), + public static ErasureCodingInfo convertECInfo(ErasureCodingInfoProto ecInfoProto) { + return new ErasureCodingInfo(ecInfoProto.getSrc(), convertECSchema(ecInfoProto.getSchema())); } - public static ECInfoProto convertECInfo(ECInfo ecInfo) { - return ECInfoProto.newBuilder().setSrc(ecInfo.getSrc()) + public static ErasureCodingInfoProto convertECInfo(ErasureCodingInfo ecInfo) { + return ErasureCodingInfoProto.newBuilder().setSrc(ecInfo.getSrc()) .setSchema(convertECSchema(ecInfo.getSchema())).build(); } @@ -3151,13 +3151,13 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { return builder.build(); } - public static ECZoneInfoProto convertECZoneInfo(ECZoneInfo ecZoneInfo) { - return ECZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir()) + public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) { + return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir()) .setSchema(convertECSchema(ecZoneInfo.getSchema())).build(); } - public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) { - return new ECZoneInfo(ecZoneInfoProto.getDir(), + public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) { + return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(), convertECSchema(ecZoneInfoProto.getSchema())); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java similarity index 97% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java index 2d634986a7db1..4c4aae9718ad1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java @@ -31,7 +31,7 @@ * This class is instantiated by the FSNamesystem. */ @InterfaceAudience.LimitedPrivate({"HDFS"}) -public final class ECSchemaManager { +public final class ErasureCodingSchemaManager { /** * TODO: HDFS-8095 @@ -55,7 +55,7 @@ public final class ECSchemaManager { */ private final Map activeSchemas; - ECSchemaManager() { + ErasureCodingSchemaManager() { this.activeSchemas = new TreeMap(); for (ECSchema schema : SYS_SCHEMAS) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 14d4e29c040e8..6b6add7497a13 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.IOException; @@ -53,11 +53,11 @@ public ErasureCodingZoneManager(FSDirectory dir) { } ECSchema getECSchema(INodesInPath iip) throws IOException { - ECZoneInfo ecZoneInfo = getECZoneInfo(iip); + ErasureCodingZoneInfo ecZoneInfo = getECZoneInfo(iip); return ecZoneInfo == null ? null : ecZoneInfo.getSchema(); } - ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { + ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { assert dir.hasReadLock(); Preconditions.checkNotNull(iip); List inodes = iip.getReadOnlyINodes(); @@ -79,9 +79,9 @@ ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { for (XAttr xAttr : xAttrs) { if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { String schemaName = new String(xAttr.getValue()); - ECSchema schema = dir.getFSNamesystem().getSchemaManager() + ECSchema schema = dir.getFSNamesystem().getECSchemaManager() .getSchema(schemaName); - return new ECZoneInfo(inode.getFullPathName(), schema); + return new ErasureCodingZoneInfo(inode.getFullPathName(), schema); } } } @@ -110,7 +110,7 @@ XAttr createErasureCodingZone(String src, ECSchema schema) // System default schema will be used since no specified. if (schema == null) { - schema = ECSchemaManager.getSystemDefaultSchema(); + schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); } // Now persist the schema name in xattr 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 8f843d5cbb91b..b830157f7a26c 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 @@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException; import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException; @@ -1250,7 +1250,7 @@ ECSchema getECSchema(INodesInPath iip) throws IOException { } } - ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { + ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { readLock(); try { return ecZoneManager.getECZoneInfo(iip); 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 79dd3d7803044..7f82d2d9686d5 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 @@ -180,8 +180,8 @@ 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.ECInfo; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -428,7 +428,7 @@ private void logAuditEvent(boolean succeeded, private final BlockManager blockManager; private final SnapshotManager snapshotManager; private final CacheManager cacheManager; - private final ECSchemaManager schemaManager; + private final ErasureCodingSchemaManager ecSchemaManager; private final DatanodeStatistics datanodeStatistics; private String nameserviceId; @@ -608,7 +608,7 @@ void clear() { leaseManager.removeAllLeases(); snapshotManager.clearSnapshottableDirs(); cacheManager.clear(); - schemaManager.clear(); + ecSchemaManager.clear(); setImageLoaded(false); blockManager.clear(); } @@ -848,7 +848,7 @@ static FSNamesystem loadFromDisk(Configuration conf) throws IOException { this.dir = new FSDirectory(this, conf); this.snapshotManager = new SnapshotManager(dir); this.cacheManager = new CacheManager(this, conf, blockManager); - this.schemaManager = new ECSchemaManager(); + this.ecSchemaManager = new ErasureCodingSchemaManager(); this.safeMode = new SafeModeInfo(conf); this.topConf = new TopConf(conf); this.auditLoggers = initAuditLoggers(conf); @@ -6632,8 +6632,8 @@ public CacheManager getCacheManager() { } /** @return the schema manager. */ - public ECSchemaManager getSchemaManager() { - return schemaManager; + public ErasureCodingSchemaManager getECSchemaManager() { + return ecSchemaManager; } @Override // NameNodeMXBean @@ -7579,11 +7579,11 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, /** * Get the erasure coding information for specified src */ - ECInfo getErasureCodingInfo(String src) throws AccessControlException, + ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException, UnresolvedLinkException, IOException { ECSchema schema = getECSchemaForPath(src); if (schema != null) { - return new ECInfo(src, schema); + return new ErasureCodingInfo(src, schema); } return null; } @@ -7591,7 +7591,7 @@ ECInfo getErasureCodingInfo(String src) throws AccessControlException, /** * Get the erasure coding zone information for specified path */ - ECZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException, + ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException, UnresolvedLinkException, IOException { checkOperation(OperationCategory.READ); final byte[][] pathComponents = FSDirectory @@ -7620,7 +7620,7 @@ ECSchema[] getECSchemas() throws IOException { readLock(); try { checkOperation(OperationCategory.READ); - return schemaManager.getSchemas(); + return ecSchemaManager.getSchemas(); } finally { readUnlock(); } @@ -7635,7 +7635,7 @@ ECSchema getECSchema(String schemaName) throws IOException { readLock(); try { checkOperation(OperationCategory.READ); - return schemaManager.getSchema(schemaName); + return ecSchemaManager.getSchema(schemaName); } finally { readUnlock(); } 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 dcf06070ace5a..5a69b2f2e0279 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 @@ -84,8 +84,8 @@ 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.ECInfo; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FSLimitException; @@ -2035,7 +2035,7 @@ public void removeSpanReceiver(long id) throws IOException { } @Override // ClientProtocol - public ECInfo getErasureCodingInfo(String src) throws IOException { + public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException { checkNNStartup(); return namesystem.getErasureCodingInfo(src); } @@ -2047,7 +2047,7 @@ public ECSchema[] getECSchemas() throws IOException { } @Override // ClientProtocol - public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException { checkNNStartup(); return namesystem.getErasureCodingZoneInfo(src); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index 802a46dd08c13..d53844da7cfca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.shell.CommandFactory; import org.apache.hadoop.fs.shell.PathData; import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.protocol.ECZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.StringUtils; @@ -164,7 +164,7 @@ protected void processPath(PathData item) throws IOException { super.processPath(item); DistributedFileSystem dfs = (DistributedFileSystem) item.fs; try { - ECZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path); + ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path); out.println(ecZoneInfo.toString()); } catch (IOException e) { throw new IOException("Unable to create EC zone for the path " 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 046120dcd5be5..0a0a4c4c934c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -867,6 +867,6 @@ service ClientNamenodeProtocol { returns(GetErasureCodingInfoResponseProto); rpc getECSchemas(GetECSchemasRequestProto) returns(GetECSchemasResponseProto); - rpc getErasureCodingZoneInfo(GetECZoneInfoRequestProto) - returns(GetECZoneInfoResponseProto); + rpc getErasureCodingZoneInfo(GetErasureCodingZoneInfoRequestProto) + returns(GetErasureCodingZoneInfoResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 702f6fd44c788..7a19a80cbf17b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -43,17 +43,17 @@ message ECSchemaProto { } /** - * ECInfo + * ErasureCodingInfo */ -message ECInfoProto { +message ErasureCodingInfoProto { required string src = 1; required ECSchemaProto schema = 2; } /** - * ECZoneInfo + * ErasureCodingZoneInfo */ -message ECZoneInfoProto { +message ErasureCodingZoneInfoProto { required string dir = 1; required ECSchemaProto schema = 2; } @@ -71,7 +71,7 @@ message GetErasureCodingInfoRequestProto { } message GetErasureCodingInfoResponseProto { - optional ECInfoProto ECInfo = 1; + optional ErasureCodingInfoProto ECInfo = 1; } message GetECSchemasRequestProto { // void request @@ -81,12 +81,12 @@ message GetECSchemasResponseProto { repeated ECSchemaProto schemas = 1; } -message GetECZoneInfoRequestProto { +message GetErasureCodingZoneInfoRequestProto { required string src = 1; // path to get the zone info } -message GetECZoneInfoResponseProto { - optional ECZoneInfoProto ECZoneInfo = 1; +message GetErasureCodingZoneInfoResponseProto { + optional ErasureCodingZoneInfoProto ECZoneInfo = 1; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index a1f704d5b1c28..4da9c26248375 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -24,7 +24,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.ECInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; @@ -34,7 +34,7 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; -import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; import org.junit.Before; @@ -54,8 +54,8 @@ public class TestDFSStripedInputStream { private DistributedFileSystem fs; private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); - private ECInfo info = new ECInfo(filePath.toString(), - ECSchemaManager.getSystemDefaultSchema()); + private ErasureCodingInfo info = new ErasureCodingInfo(filePath.toString(), + ErasureCodingSchemaManager.getSystemDefaultSchema()); private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java index 83d208a1a0bf3..88198c914a045 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.After; import org.junit.Before; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index f1aec821b9ff2..59818c09c33a8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -20,8 +20,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.protocol.ECInfo; -import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -158,7 +158,7 @@ public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { assertNull(fs.getClient().getErasureCodingInfo(src)); // dir ECInfo after creating ec zone fs.getClient().createErasureCodingZone(src, null); //Default one will be used. - ECSchema sysDefaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); verifyErasureCodingInfo(src, sysDefaultSchema); fs.create(new Path(ecDir, "/child1")).close(); // verify for the files in ec zone @@ -167,7 +167,7 @@ public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { @Test public void testGetErasureCodingInfo() throws Exception { - ECSchema[] sysSchemas = ECSchemaManager.getSystemSchemas(); + ECSchema[] sysSchemas = ErasureCodingSchemaManager.getSystemSchemas(); assertTrue("System schemas should be of only 1 for now", sysSchemas.length == 1); @@ -187,7 +187,7 @@ public void testGetErasureCodingInfo() throws Exception { private void verifyErasureCodingInfo( String src, ECSchema usingSchema) throws IOException { - ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src); + ErasureCodingInfo ecInfo = fs.getClient().getErasureCodingInfo(src); assertNotNull("ECInfo should have been non-null", ecInfo); assertEquals(src, ecInfo.getSrc()); ECSchema schema = ecInfo.getSchema(); 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 f580cbbebd01e..1be00b6fd8142 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 @@ -71,7 +71,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; -import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; @@ -663,7 +663,7 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices0 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo( new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0, - liveBlkIndices0, ECSchemaManager.getSystemDefaultSchema()); + liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema()); DatanodeInfo[] dnInfos1 = new DatanodeInfo[] { DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() }; DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil @@ -677,7 +677,7 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices1 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo( new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1, - liveBlkIndices1, ECSchemaManager.getSystemDefaultSchema()); + liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema()); List blkRecoveryInfosList = new ArrayList(); blkRecoveryInfosList.add(blkECRecoveryInfo0); blkRecoveryInfosList.add(blkECRecoveryInfo1); @@ -723,8 +723,8 @@ private void assertBlockECRecoveryInfoEquals( ECSchema ecSchema2 = blkECRecoveryInfo2.getECSchema(); // Compare ECSchemas same as default ECSchema as we used system default // ECSchema used in this test - compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema1); - compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema2); + compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema1); + compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema2); } private void compareECSchemas(ECSchema ecSchema1, ECSchema ecSchema2) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index f97943b58ed0b..7a330b9c48a88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -60,7 +60,7 @@ public void testBlockStripedFeature() @Test public void testBlockStripedTotalBlockCount() { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped = new BlockInfoStriped(blk, @@ -72,7 +72,7 @@ public void testBlockStripedTotalBlockCount() { @Test public void testBlockStripedLength() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); @@ -87,7 +87,7 @@ public void testBlockStripedLength() @Test public void testBlockStripedConsumedSpace() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); @@ -116,7 +116,7 @@ public void testBlockStripedConsumedSpace() @Test public void testMultipleBlockStripedConsumedSpace() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk1 = new Block(1); @@ -141,7 +141,7 @@ public void testMultipleBlockStripedConsumedSpace() @Test public void testBlockStripedFileSize() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); @@ -160,7 +160,7 @@ public void testBlockStripedFileSize() @Test public void testBlockStripedUCFileSize() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); @@ -177,7 +177,7 @@ public void testBlockStripedUCFileSize() @Test public void testBlockStripedComputeQuotaUsage() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); @@ -204,7 +204,7 @@ public void testBlockStripedComputeQuotaUsage() @Test public void testBlockStripedUCComputeQuotaUsage() throws IOException, InterruptedException { - ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema(); + ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); From a17cedb44c2a98d863407fab95c2b7f0893d0727 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 7 May 2015 11:06:40 -0700 Subject: [PATCH 103/212] HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream. Contributed by Yi Liu. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 88 ++++++++++++++++--- .../hadoop/hdfs/TestWriteReadStripedFile.java | 83 ++++++++++++++--- 3 files changed, 151 insertions(+), 23 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 11e83760226be..fed08e1797157 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -186,3 +186,6 @@ HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding (umamahesh) + + HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream. + (Yi Liu via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 7cb7b6dd8603d..9011192aab64d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -19,10 +19,13 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.fs.ChecksumException; +import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.ByteBufferPool; + import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions; @@ -31,9 +34,11 @@ import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; +import java.io.EOFException; import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.util.EnumSet; import java.util.Set; import java.util.Map; import java.util.HashMap; @@ -263,6 +268,10 @@ protected void closeCurrentBlockReaders() { } private long getOffsetInBlockGroup() { + return getOffsetInBlockGroup(pos); + } + + private long getOffsetInBlockGroup(long pos) { return pos - currentLocatedBlock.getStartOffset(); } @@ -278,18 +287,22 @@ private void readOneStripe( // compute stripe range based on pos final long offsetInBlockGroup = getOffsetInBlockGroup(); final long stripeLen = cellSize * dataBlkNum; - int stripeIndex = (int) (offsetInBlockGroup / stripeLen); - curStripeRange = new StripeRange(stripeIndex * stripeLen, - Math.min(currentLocatedBlock.getBlockSize() - (stripeIndex * stripeLen), - stripeLen)); - final int numCell = (int) ((curStripeRange.length - 1) / cellSize + 1); + final int stripeIndex = (int) (offsetInBlockGroup / stripeLen); + final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen); + final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize() + - (stripeIndex * stripeLen), stripeLen); + curStripeRange = new StripeRange(offsetInBlockGroup, + stripeLimit - stripeBufOffset); + + final int startCell = stripeBufOffset / cellSize; + final int numCell = (stripeLimit - 1) / cellSize + 1; // read the whole stripe in parallel Map, Integer> futures = new HashMap<>(); - for (int i = 0; i < numCell; i++) { - curStripeBuf.position(cellSize * i); - curStripeBuf.limit((int) Math.min(cellSize * (i + 1), - curStripeRange.length)); + for (int i = startCell; i < numCell; i++) { + int bufPos = i == startCell ? stripeBufOffset : cellSize * i; + curStripeBuf.position(bufPos); + curStripeBuf.limit(Math.min(cellSize * (i + 1), stripeLimit)); ByteBuffer buf = curStripeBuf.slice(); ByteBufferStrategy strategy = new ByteBufferStrategy(buf); final int targetLength = buf.remaining(); @@ -329,6 +342,39 @@ public Integer call() throws Exception { }; } + /** + * Seek to a new arbitrary location + */ + @Override + public synchronized void seek(long targetPos) throws IOException { + if (targetPos > getFileLength()) { + throw new EOFException("Cannot seek after EOF"); + } + if (targetPos < 0) { + throw new EOFException("Cannot seek to negative offset"); + } + if (closed.get()) { + throw new IOException("Stream is closed!"); + } + if (targetPos <= blockEnd) { + final long targetOffsetInBlk = getOffsetInBlockGroup(targetPos); + if (curStripeRange.include(targetOffsetInBlk)) { + int bufOffset = getStripedBufOffset(targetOffsetInBlk); + curStripeBuf.position(bufOffset); + pos = targetPos; + return; + } + } + pos = targetPos; + blockEnd = -1; + } + + private int getStripedBufOffset(long offsetInBlockGroup) { + final long stripeLen = cellSize * dataBlkNum; + // compute the position in the curStripeBuf based on "pos" + return (int) (offsetInBlockGroup % stripeLen); + } + @Override protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException { @@ -405,10 +451,8 @@ private int readBuffer(BlockReader blockReader, * @return number of bytes copied */ private int copy(ReaderStrategy strategy, int offset, int length) { - final long stripeLen = cellSize * dataBlkNum; - final long offsetInBlk = pos - currentLocatedBlock.getStartOffset(); - // compute the position in the curStripeBuf based on "pos" - int bufOffset = (int) (offsetInBlk % stripeLen); + final long offsetInBlk = getOffsetInBlockGroup(); + int bufOffset = getStripedBufOffset(offsetInBlk); curStripeBuf.position(bufOffset); return strategy.copyFrom(curStripeBuf, offset, Math.min(length, curStripeBuf.remaining())); @@ -546,4 +590,22 @@ private void waitNextCompletion(CompletionService service, } throw new InterruptedException("let's retry"); } + + /** + * May need online read recovery, zero-copy read doesn't make + * sense, so don't support it. + */ + @Override + public synchronized ByteBuffer read(ByteBufferPool bufferPool, + int maxLength, EnumSet opts) + throws IOException, UnsupportedOperationException { + throw new UnsupportedOperationException( + "Not support enhanced byte buffer access."); + } + + @Override + public synchronized void releaseBuffer(ByteBuffer buffer) { + throw new UnsupportedOperationException( + "Not support enhanced byte buffer access."); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index eacc6edddf636..5c6f449f5f75e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -22,12 +22,12 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.HdfsConstants; - import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; @@ -150,11 +150,35 @@ private byte[] generateBytes(int cnt) { return bytes; } + private int readAll(FSDataInputStream in, byte[] buf) throws IOException { + int readLen = 0; + int ret; + do { + ret = in.read(buf, readLen, buf.length - readLen); + if (ret > 0) { + readLen += ret; + } + } while (ret >= 0 && readLen < buf.length); + return readLen; + } + private byte getByte(long pos) { final int mod = 29; return (byte) (pos % mod + 1); } + private void assertSeekAndRead(FSDataInputStream fsdis, int pos, + int writeBytes) throws IOException { + fsdis.seek(pos); + byte[] buf = new byte[writeBytes]; + int readLen = readAll(fsdis, buf); + Assert.assertEquals(readLen, writeBytes - pos); + for (int i = 0; i < readLen; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", + getByte(pos + i), buf[i]); + } + } + private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) throws IOException { Path testPath = new Path(src); @@ -183,15 +207,7 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) // stateful read with byte array try (FSDataInputStream fsdis = fs.open(new Path(src))) { byte[] buf = new byte[writeBytes + 100]; - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf, readLen, buf.length - readLen); - if (ret > 0) { - readLen += ret; - } - } while (ret >= 0); - readLen = readLen >= 0 ? readLen : 0; + int readLen = readAll(fsdis, buf); Assert.assertEquals("The length of file should be the same to write size", writeBytes, readLen); for (int i = 0; i < writeBytes; i++) { @@ -200,6 +216,53 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) } } + // seek and stateful read + try (FSDataInputStream fsdis = fs.open(new Path(src))) { + // seek to 1/2 of content + int pos = writeBytes/2; + assertSeekAndRead(fsdis, pos, writeBytes); + + // seek to 1/3 of content + pos = writeBytes/3; + assertSeekAndRead(fsdis, pos, writeBytes); + + // seek to 0 pos + pos = 0; + assertSeekAndRead(fsdis, pos, writeBytes); + + if (writeBytes > cellSize) { + // seek to cellSize boundary + pos = cellSize -1; + assertSeekAndRead(fsdis, pos, writeBytes); + } + + if (writeBytes > cellSize * dataBlocks) { + // seek to striped cell group boundary + pos = cellSize * dataBlocks - 1; + assertSeekAndRead(fsdis, pos, writeBytes); + } + + if (writeBytes > blockSize * dataBlocks) { + // seek to striped block group boundary + pos = blockSize * dataBlocks - 1; + assertSeekAndRead(fsdis, pos, writeBytes); + } + + try { + fsdis.seek(-1); + Assert.fail("Should be failed if seek to negative offset"); + } catch (EOFException e) { + // expected + } + + try { + fsdis.seek(writeBytes + 1); + Assert.fail("Should be failed if seek after EOF"); + } catch (EOFException e) { + // expected + } + } + // stateful read with ByteBuffer try (FSDataInputStream fsdis = fs.open(new Path(src))) { ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100); From 9da927540f0ea6698388a4e79ef32c4dc51495ea Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 7 May 2015 11:52:49 -0700 Subject: [PATCH 104/212] HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. Contributed by Yong Zhang. --- .../hadoop/hdfs/protocol/HdfsFileStatus.java | 10 ++- .../SnapshottableDirectoryStatus.java | 2 +- .../hadoop/hdfs/web/JsonUtilClient.java | 2 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../org/apache/hadoop/hdfs/DFSClient.java | 6 +- .../apache/hadoop/hdfs/DFSOutputStream.java | 2 +- .../hadoop/hdfs/DFSStripedInputStream.java | 13 ++-- .../hadoop/hdfs/DFSStripedOutputStream.java | 4 +- .../hdfs/protocol/HdfsLocatedFileStatus.java | 5 +- .../ClientNamenodeProtocolTranslatorPB.java | 2 +- .../hadoop/hdfs/protocolPB/PBHelper.java | 10 ++- .../namenode/FSDirStatAndListingOp.java | 16 +++-- .../src/main/proto/erasurecoding.proto | 19 ------ .../hadoop-hdfs/src/main/proto/hdfs.proto | 22 +++++++ .../hadoop/hdfs/TestDFSClientRetries.java | 4 +- .../hdfs/TestDFSStripedInputStream.java | 16 ++--- .../hadoop/hdfs/TestEncryptionZones.java | 2 +- .../hdfs/TestFileStatusWithECschema.java | 65 +++++++++++++++++++ .../org/apache/hadoop/hdfs/TestLease.java | 4 +- .../hadoop/hdfs/server/namenode/TestFsck.java | 2 +- .../apache/hadoop/hdfs/web/TestJsonUtil.java | 2 +- 21 files changed, 149 insertions(+), 62 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java index 34f429a21b517..f07973a8e20ea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtilClient; +import org.apache.hadoop.io.erasurecode.ECSchema; /** Interface that represents the over the wire information for a file. */ @@ -48,6 +49,8 @@ public class HdfsFileStatus { private final FileEncryptionInfo feInfo; + private final ECSchema schema; + // Used by dir, not including dot and dotdot. Always zero for a regular file. private final int childrenNum; private final byte storagePolicy; @@ -73,7 +76,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo, - byte storagePolicy) { + byte storagePolicy, ECSchema schema) { this.length = length; this.isdir = isdir; this.block_replication = (short)block_replication; @@ -93,6 +96,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, this.childrenNum = childrenNum; this.feInfo = feInfo; this.storagePolicy = storagePolicy; + this.schema = schema; } /** @@ -250,6 +254,10 @@ public final FileEncryptionInfo getFileEncryptionInfo() { return feInfo; } + public ECSchema getECSchema() { + return schema; + } + public final int getChildrenNum() { return childrenNum; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java index ac19d44cd4253..813ea266cd07f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java @@ -61,7 +61,7 @@ public SnapshottableDirectoryStatus(long modification_time, long access_time, int snapshotNumber, int snapshotQuota, byte[] parentFullPath) { this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time, access_time, permission, owner, group, null, localName, inodeId, - childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED); + childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); this.snapshotNumber = snapshotNumber; this.snapshotQuota = snapshotQuota; this.parentFullPath = parentFullPath; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java index ca94840973705..62f679b727a87 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java @@ -132,7 +132,7 @@ static HdfsFileStatus toFileStatus(final Map json, boolean includesType) { blockSize, mTime, aTime, permission, owner, group, symlink, DFSUtilClient.string2Bytes(localName), fileId, childrenNum, null, - storagePolicy); + storagePolicy, null); } /** Convert a Json map to an ExtendedBlock object. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index fed08e1797157..ab8a748ab1497 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -189,3 +189,6 @@ HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream. (Yi Liu via jing9) + + HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. (Yong Zhang via + jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 8f250fdd34249..9155b4d09ec63 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1193,9 +1193,9 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) // Get block info from namenode TraceScope scope = getPathTraceScope("newDFSInputStream", src); try { - ErasureCodingInfo info = getErasureCodingInfo(src); - if (info != null) { - return new DFSStripedInputStream(this, src, verifyChecksum, info); + ECSchema schema = getFileInfo(src).getECSchema(); + if (schema != null) { + return new DFSStripedInputStream(this, src, verifyChecksum, schema); } else { return new DFSInputStream(this, src, verifyChecksum); } 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 8580357d8bfe7..ea1ea269da581 100755 --- 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 @@ -271,7 +271,7 @@ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, } Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!"); final DFSOutputStream out; - if(stat.getReplication() == 0) { + if(stat.getECSchema() != null) { out = new DFSStripedOutputStream(dfsClient, src, stat, flag, progress, checksum, favoredNodes); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 9011192aab64d..7425e753441ae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -29,6 +29,7 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.NetUtils; import org.apache.htrace.Span; import org.apache.htrace.Trace; @@ -132,13 +133,13 @@ boolean include(long pos) { private final CompletionService readingService; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, - ErasureCodingInfo ecInfo) throws IOException { + ECSchema schema) throws IOException { super(dfsClient, src, verifyChecksum); - // ECInfo is restored from NN just before reading striped file. - assert ecInfo != null; - cellSize = ecInfo.getSchema().getChunkSize(); - dataBlkNum = (short) ecInfo.getSchema().getNumDataUnits(); - parityBlkNum = (short) ecInfo.getSchema().getNumParityUnits(); + + assert schema != null; + cellSize = schema.getChunkSize(); + dataBlkNum = (short) schema.getNumDataUnits(); + parityBlkNum = (short) schema.getNumParityUnits(); curStripeRange = new StripeRange(0, 0); readingService = new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index bbc8ba0beba69..b99afab954703 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -219,9 +219,7 @@ private StripedDataStreamer getLeadingStreamer() { LOG.debug("Creating DFSStripedOutputStream for " + src); } - // ECInfo is restored from NN just before writing striped files. - //TODO reduce an rpc call HDFS-8289 - final ECSchema schema = dfsClient.getErasureCodingInfo(src).getSchema(); + final ECSchema schema = stat.getECSchema(); final int numParityBlocks = schema.getNumParityUnits(); cellSize = schema.getChunkSize(); numDataBlocks = schema.getNumDataUnits(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java index 23e8f57839bf1..9194d26258d74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtilClient; +import org.apache.hadoop.io.erasurecode.ECSchema; /** * Interface that represents the over the wire information @@ -58,10 +59,10 @@ public HdfsLocatedFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, LocatedBlocks locations, - int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) { + int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy, ECSchema schema) { super(length, isdir, block_replication, blocksize, modification_time, access_time, permission, owner, group, symlink, path, fileId, - childrenNum, feInfo, storagePolicy); + childrenNum, feInfo, storagePolicy, schema); this.locations = locations; } 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 014fcef80bf91..67b1457414cd0 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 @@ -173,7 +173,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; 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 26bdf3480ff07..94b2ff9013e67 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 @@ -136,9 +136,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; 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.BlockStoragePolicyProto; @@ -1505,7 +1505,8 @@ public static HdfsFileStatus convert(HdfsFileStatusProto fs) { fs.hasChildrenNum() ? fs.getChildrenNum() : -1, fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null, fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy() - : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED); + : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, + fs.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null); } public static SnapshottableDirectoryStatus convert( @@ -1566,6 +1567,9 @@ public static HdfsFileStatusProto convert(HdfsFileStatus fs) { builder.setLocations(PBHelper.convert(locations)); } } + if(fs.getECSchema() != null) { + builder.setEcSchema(PBHelper.convertECSchema(fs.getECSchema())); + } return builder.build(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index c636d933ac3cc..7133cf1950be4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode; import com.google.common.base.Preconditions; + import org.apache.commons.io.Charsets; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException; @@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.util.ReadOnlyList; +import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.FileNotFoundException; import java.io.IOException; @@ -315,7 +317,7 @@ static HdfsFileStatus getFileInfo( if (fsd.getINode4DotSnapshot(srcs) != null) { return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null, HdfsFileStatus.EMPTY_NAME, -1L, 0, null, - HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED); + HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); } return null; } @@ -382,7 +384,9 @@ static HdfsFileStatus createFileStatus( final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); - + + final ECSchema schema = fsd.getECSchema(iip); + if (node.isFile()) { final INodeFile fileNode = node.asFile(); size = fileNode.computeFileSize(snapshot); @@ -412,7 +416,8 @@ static HdfsFileStatus createFileStatus( node.getId(), childrenNum, feInfo, - storagePolicy); + storagePolicy, + schema); } private static INodeAttributes getINodeAttributes( @@ -459,7 +464,8 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - + final ECSchema schema = fsd.getECSchema(iip); + HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(size, node.isDirectory(), replication, blocksize, node.getModificationTime(snapshot), @@ -467,7 +473,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( getPermissionForFileStatus(nodeAttrs, isEncrypted), nodeAttrs.getUserName(), nodeAttrs.getGroupName(), node.isSymlink() ? node.asSymlink().getSymlink() : null, path, - node.getId(), loc, childrenNum, feInfo, storagePolicy); + node.getId(), loc, childrenNum, feInfo, storagePolicy, schema); // Set caching information for the located blocks. if (loc != null) { CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 7a19a80cbf17b..2302d1d4c4c5b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -23,25 +23,6 @@ package hadoop.hdfs; import "hdfs.proto"; -/** - * ECSchema options entry - */ -message ECSchemaOptionEntryProto { - required string key = 1; - required string value = 2; -} - -/** - * ECSchema for erasurecoding - */ -message ECSchemaProto { - required string schemaName = 1; - required string codecName = 2; - required uint32 dataUnits = 3; - required uint32 parityUnits = 4; - repeated ECSchemaOptionEntryProto options = 5; -} - /** * ErasureCodingInfo */ 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 67e20582f2230..64030bebd37f0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -303,6 +303,25 @@ message LocatedBlocksProto { optional FileEncryptionInfoProto fileEncryptionInfo = 6; } +/** + * ECSchema options entry + */ +message ECSchemaOptionEntryProto { + required string key = 1; + required string value = 2; +} + +/** + * ECSchema for erasurecoding + */ +message ECSchemaProto { + required string schemaName = 1; + required string codecName = 2; + required uint32 dataUnits = 3; + required uint32 parityUnits = 4; + repeated ECSchemaOptionEntryProto options = 5; +} + /** * Status of a file, directory or symlink * Optionally includes a file's block locations if requested by client on the rpc call. @@ -337,6 +356,9 @@ message HdfsFileStatusProto { optional FileEncryptionInfoProto fileEncryptionInfo = 15; optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id + + // Optional field for erasure coding + optional ECSchemaProto ecSchema = 17; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java index 68cc1552aa9e4..ec88a542de45b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java @@ -255,12 +255,12 @@ public Object answer(InvocationOnMock invocation) Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)).when(mockNN).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString()); Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)) + 1010, 0, null, (byte) 0, null)) .when(mockNN) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 4da9c26248375..3f79933a32961 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -24,7 +24,6 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; @@ -36,6 +35,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -54,8 +54,7 @@ public class TestDFSStripedInputStream { private DistributedFileSystem fs; private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); - private ErasureCodingInfo info = new ErasureCodingInfo(filePath.toString(), - ErasureCodingSchemaManager.getSystemDefaultSchema()); + private final ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @@ -92,8 +91,8 @@ public void testGetBlock() throws Exception { NUM_STRIPE_PER_BLOCK, false); LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); - final DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info); + final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), + filePath.toString(), false, schema); List lbList = lbs.getLocatedBlocks(); for (LocatedBlock aLbList : lbList) { @@ -129,7 +128,7 @@ public void testPread() throws Exception { } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, info); + filePath.toString(), false, schema); int readSize = BLOCK_GROUP_SIZE; byte[] readBuffer = new byte[readSize]; int ret = in.read(0, readBuffer, 0, readSize); @@ -156,8 +155,7 @@ private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) } DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, NUM_STRIPE_PER_BLOCK, false); - LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, fileSize); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(filePath.toString(), 0, fileSize); assert lbs.getLocatedBlocks().size() == numBlocks; for (LocatedBlock lb : lbs.getLocatedBlocks()) { @@ -175,7 +173,7 @@ private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), - false, info); + false, schema); byte[] expected = new byte[fileSize]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java index e0bd6f44cc898..6e2ec5ec95354 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java @@ -737,7 +737,7 @@ private static void mockCreate(ClientProtocol mcp, version, new byte[suite.getAlgorithmBlockSize()], new byte[suite.getAlgorithmBlockSize()], "fakeKey", "fakeVersion"), - (byte) 0)) + (byte) 0, null)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java new file mode 100644 index 0000000000000..f8c0667eb00e7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java @@ -0,0 +1,65 @@ +package org.apache.hadoop.hdfs; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestFileStatusWithECschema { + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private DFSClient client; + + @Before + public void before() throws IOException { + cluster = + new MiniDFSCluster.Builder(new Configuration()).numDataNodes(1).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + client = fs.getClient(); + } + + @After + public void after() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testFileStatusWithECschema() throws Exception { + // test directory not in EC zone + final Path dir = new Path("/foo"); + assertTrue(fs.mkdir(dir, FsPermission.getDirDefault())); + assertNull(client.getFileInfo(dir.toString()).getECSchema()); + // test file not in EC zone + final Path file = new Path(dir, "foo"); + fs.create(file).close(); + assertNull(client.getFileInfo(file.toString()).getECSchema()); + fs.delete(file, true); + + final ECSchema schema1 = ErasureCodingSchemaManager.getSystemDefaultSchema(); + // create EC zone on dir + fs.createErasureCodingZone(dir, schema1); + final ECSchema schame2 = client.getFileInfo(dir.toUri().getPath()).getECSchema(); + assertNotNull(schame2); + assertTrue(schema1.equals(schame2)); + + // test file in EC zone + fs.create(file).close(); + final ECSchema schame3 = + fs.getClient().getFileInfo(file.toUri().getPath()).getECSchema(); + assertNotNull(schame3); + assertTrue(schema1.equals(schame3)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java index 985f43ee40b4e..b77ff3a000668 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java @@ -354,12 +354,12 @@ public void testFactory() throws Exception { Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)).when(mcp).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString()); Mockito .doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)) + 1010, 0, null, (byte) 0, null)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), 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 eabd0c8c938e3..787082791a6ca 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 @@ -1198,7 +1198,7 @@ public void testFsckFileNotFound() throws Exception { HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication, blockSize, modTime, accessTime, perms, owner, group, symlink, - path, fileId, numChildren, null, storagePolicy); + path, fileId, numChildren, null, storagePolicy, null); Result res = new Result(conf); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java index 391f1903f96c1..8947c5b49a16d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java @@ -65,7 +65,7 @@ public void testHdfsFileStatus() throws IOException { final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26, now, now + 10, new FsPermission((short) 0644), "user", "group", DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"), - HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0); + HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null); final FileStatus fstatus = toFileStatus(status, parent); System.out.println("status = " + status); System.out.println("fstatus = " + fstatus); From 51ea117f883f9c049de58987dc66e07e71a68ee4 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 8 May 2015 13:56:56 -0700 Subject: [PATCH 105/212] HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction. Contributed by Tsz Wo Nicholas Sze. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockInfo.java | 95 +------------------ .../BlockInfoContiguousUnderConstruction.java | 27 ++---- .../BlockInfoStripedUnderConstruction.java | 25 ++--- .../BlockInfoUnderConstruction.java | 27 ++++++ .../server/blockmanagement/BlockManager.java | 51 +++++++--- .../hdfs/server/namenode/FSNamesystem.java | 20 ++-- 7 files changed, 95 insertions(+), 153 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index ab8a748ab1497..c7d01c70c17ad 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -192,3 +192,6 @@ HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. (Yong Zhang via jing9) + + HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction. + (Tsz Wo Nicholas Sze via jing9) 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 8b7192521aaa2..aebfbb185639f 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 @@ -17,13 +17,12 @@ */ package org.apache.hadoop.hdfs.server.blockmanagement; +import java.util.LinkedList; + import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.util.LightWeightGSet; -import java.io.IOException; -import java.util.LinkedList; - /** * For a given block (or an erasure coding block group), BlockInfo class * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes @@ -336,94 +335,4 @@ public LightWeightGSet.LinkedElement getNext() { public void setNext(LightWeightGSet.LinkedElement next) { this.nextLinkedElement = next; } - - static BlockInfo copyOf(BlockInfo b) { - if (!b.isStriped()) { - return new BlockInfoContiguous((BlockInfoContiguous) b); - } else { - return new BlockInfoStriped((BlockInfoStriped) b); - } - } - - static BlockInfo convertToCompleteBlock(BlockInfo blk) throws IOException { - if (blk instanceof BlockInfoContiguousUnderConstruction) { - return ((BlockInfoContiguousUnderConstruction) blk) - .convertToCompleteBlock(); - } else if (blk instanceof BlockInfoStripedUnderConstruction) { - return ((BlockInfoStripedUnderConstruction) blk).convertToCompleteBlock(); - } else { - return blk; - } - } - - static void commitBlock(BlockInfo blockInfo, Block reported) - throws IOException { - if (blockInfo instanceof BlockInfoContiguousUnderConstruction) { - ((BlockInfoContiguousUnderConstruction) blockInfo).commitBlock(reported); - } else if (blockInfo instanceof BlockInfoStripedUnderConstruction) { - ((BlockInfoStripedUnderConstruction) blockInfo).commitBlock(reported); - } - } - - static void addReplica(BlockInfo ucBlock, DatanodeStorageInfo storageInfo, - Block reportedBlock, HdfsServerConstants.ReplicaState reportedState) { - assert ucBlock instanceof BlockInfoContiguousUnderConstruction || - ucBlock instanceof BlockInfoStripedUnderConstruction; - if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { - ((BlockInfoContiguousUnderConstruction) ucBlock).addReplicaIfNotPresent( - storageInfo, reportedBlock, reportedState); - } else { // StripedUC - ((BlockInfoStripedUnderConstruction) ucBlock).addReplicaIfNotPresent( - storageInfo, reportedBlock, reportedState); - } - } - - static int getNumExpectedLocations(BlockInfo ucBlock) { - assert ucBlock instanceof BlockInfoContiguousUnderConstruction || - ucBlock instanceof BlockInfoStripedUnderConstruction; - if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { - return ((BlockInfoContiguousUnderConstruction) ucBlock) - .getNumExpectedLocations(); - } else { // StripedUC - return ((BlockInfoStripedUnderConstruction) ucBlock) - .getNumExpectedLocations(); - } - } - - public static DatanodeStorageInfo[] getExpectedStorageLocations( - BlockInfo ucBlock) { - assert ucBlock instanceof BlockInfoContiguousUnderConstruction || - ucBlock instanceof BlockInfoStripedUnderConstruction; - if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { - return ((BlockInfoContiguousUnderConstruction) ucBlock) - .getExpectedStorageLocations(); - } else { // StripedUC - return ((BlockInfoStripedUnderConstruction) ucBlock) - .getExpectedStorageLocations(); - } - } - - public static void setExpectedLocations(BlockInfo ucBlock, - DatanodeStorageInfo[] targets) { - assert ucBlock instanceof BlockInfoContiguousUnderConstruction || - ucBlock instanceof BlockInfoStripedUnderConstruction; - if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { - ((BlockInfoContiguousUnderConstruction) ucBlock) - .setExpectedLocations(targets); - } else { // StripedUC - ((BlockInfoStripedUnderConstruction) ucBlock) - .setExpectedLocations(targets); - } - } - - public static long getBlockRecoveryId(BlockInfo ucBlock) { - assert ucBlock instanceof BlockInfoContiguousUnderConstruction || - ucBlock instanceof BlockInfoStripedUnderConstruction; - if (ucBlock instanceof BlockInfoContiguousUnderConstruction) { - return ((BlockInfoContiguousUnderConstruction) ucBlock) - .getBlockRecoveryId(); - } else { // StripedUC - return ((BlockInfoStripedUnderConstruction) ucBlock).getBlockRecoveryId(); - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java index 9ba2978ac7be9..ce2219a6a1535 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java @@ -80,16 +80,8 @@ assert getBlockUCState() != BlockUCState.COMPLETE : setExpectedLocations(targets); } - /** - * Convert an under construction block to a complete block. - * - * @return BlockInfoContiguous - a complete block. - * @throws IOException if the state of the block - * (the generation stamp and the length) has not been committed by - * the client or it does not have at least a minimal number of replicas - * reported from data-nodes. - */ - BlockInfoContiguous convertToCompleteBlock() throws IOException { + @Override + public BlockInfoContiguous convertToCompleteBlock() throws IOException { assert getBlockUCState() != BlockUCState.COMPLETE : "Trying to convert a COMPLETE block"; return new BlockInfoContiguous(this); @@ -170,13 +162,8 @@ public void setGenerationStampAndVerifyReplicas(long genStamp) { } } - /** - * Commit block's length and generation stamp as reported by the client. - * Set block state to {@link BlockUCState#COMMITTED}. - * @param block - contains client reported block length and generation - * @throws IOException if block ids are inconsistent. - */ - void commitBlock(Block block) throws IOException { + @Override + public void commitBlock(Block block) throws IOException { if(getBlockId() != block.getBlockId()) throw new IOException("Trying to commit inconsistent block: id = " + block.getBlockId() + ", expected id = " + getBlockId()); @@ -235,9 +222,9 @@ public void initializeBlockRecovery(long recoveryId) { } } - void addReplicaIfNotPresent(DatanodeStorageInfo storage, - Block block, - ReplicaState rState) { + @Override + public void addReplicaIfNotPresent(DatanodeStorageInfo storage, + Block block, ReplicaState rState) { Iterator it = replicas.iterator(); while (it.hasNext()) { ReplicaUnderConstruction r = it.next(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index 0373314abb987..b9916156edacc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -73,16 +73,8 @@ assert getBlockUCState() != COMPLETE : setExpectedLocations(targets); } - /** - * Convert an under construction striped block to a complete striped block. - * - * @return BlockInfoStriped - a complete block. - * @throws IOException if the state of the block - * (the generation stamp and the length) has not been committed by - * the client or it does not have at least a minimal number of replicas - * reported from data-nodes. - */ - BlockInfoStriped convertToCompleteBlock() throws IOException { + @Override + public BlockInfoStriped convertToCompleteBlock() throws IOException { assert getBlockUCState() != COMPLETE : "Trying to convert a COMPLETE block"; return new BlockInfoStriped(this); @@ -177,12 +169,8 @@ public void setGenerationStampAndVerifyReplicas(long genStamp) { } } - /** - * Commit block's length and generation stamp as reported by the client. - * Set block state to {@link BlockUCState#COMMITTED}. - * @param block - contains client reported block length and generation - */ - void commitBlock(Block block) throws IOException { + @Override + public void commitBlock(Block block) throws IOException { if (getBlockId() != block.getBlockId()) { throw new IOException("Trying to commit inconsistent block: id = " + block.getBlockId() + ", expected id = " + getBlockId()); @@ -242,8 +230,9 @@ public void initializeBlockRecovery(long recoveryId) { } } - void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock, - ReplicaState rState) { + @Override + public void addReplicaIfNotPresent(DatanodeStorageInfo storage, + Block reportedBlock, ReplicaState rState) { if (replicas == null) { replicas = new ReplicaUnderConstruction[1]; replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState); 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 bfdd3864a5ecb..10a8caeae3c4e 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 @@ -17,7 +17,11 @@ */ package org.apache.hadoop.hdfs.server.blockmanagement; +import java.io.IOException; + import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; public interface BlockInfoUnderConstruction { /** @@ -54,4 +58,27 @@ public interface BlockInfoUnderConstruction { * make it primary. */ public void initializeBlockRecovery(long recoveryId); + + /** Add the reported replica if it is not already in the replica list. */ + public void addReplicaIfNotPresent(DatanodeStorageInfo storage, + Block reportedBlock, ReplicaState rState); + + /** + * Commit block's length and generation stamp as reported by the client. + * Set block state to {@link BlockUCState#COMMITTED}. + * @param block - contains client reported block length and generation + * @throws IOException if block ids are inconsistent. + */ + public void commitBlock(Block block) throws IOException; + + /** + * Convert an under construction block to a complete block. + * + * @return a complete block. + * @throws IOException + * if the state of the block (the generation stamp and the length) + * has not been committed by the client or it does not have at least + * a minimal number of replicas reported from data-nodes. + */ + public BlockInfo convertToCompleteBlock() throws IOException; } 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 a3d75b5ae74f1..aa0343a3a1b4f 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 @@ -637,13 +637,19 @@ public boolean hasMinStorage(BlockInfo block, int liveNum) { */ private static boolean commitBlock(final BlockInfo block, final Block commitBlock) throws IOException { - if (block.getBlockUCState() == BlockUCState.COMMITTED) - return false; - assert block.getNumBytes() <= commitBlock.getNumBytes() : - "commitBlock length is less than the stored one " - + commitBlock.getNumBytes() + " vs. " + block.getNumBytes(); - BlockInfo.commitBlock(block, commitBlock); - return true; + if (block instanceof BlockInfoUnderConstruction + && block.getBlockUCState() != BlockUCState.COMMITTED) { + final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block; + + assert block.getNumBytes() <= commitBlock.getNumBytes() : + "commitBlock length is less than the stored one " + + commitBlock.getNumBytes() + " vs. " + block.getNumBytes(); + + uc.commitBlock(commitBlock); + return true; + } + + return false; } /** @@ -700,7 +706,10 @@ private BlockInfo completeBlock(final BlockCollection bc, "Cannot complete block: block has not been COMMITTED by the client"); } - final BlockInfo completeBlock = BlockInfo.convertToCompleteBlock(curBlock); + final BlockInfo completeBlock + = !(curBlock instanceof BlockInfoUnderConstruction)? curBlock + : ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock(); + // replace penultimate block in file bc.setBlock(blkIndex, completeBlock); @@ -738,7 +747,9 @@ private BlockInfo completeBlock(final BlockCollection bc, */ public BlockInfo forceCompleteBlock(final BlockCollection bc, final BlockInfo block) throws IOException { - BlockInfo.commitBlock(block, block); + if (block instanceof BlockInfoUnderConstruction) { + ((BlockInfoUnderConstruction)block).commitBlock(block); + } return completeBlock(bc, block, true); } @@ -2250,12 +2261,13 @@ private void processFirstBlockReport( // If block is under construction, add this replica to its list if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) { - BlockInfo.addReplica(storedBlock, storageInfo, iblk, reportedState); + final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)storedBlock; + uc.addReplicaIfNotPresent(storageInfo, 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 if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) { - int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock); + int numOfReplicas = uc.getNumExpectedLocations(); namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock); } //and fall through to next clause @@ -2617,7 +2629,8 @@ private boolean isBlockUnderConstruction(BlockInfo storedBlock, void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, DatanodeStorageInfo storageInfo) throws IOException { BlockInfo block = ucBlock.storedBlock; - BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, + final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block; + uc.addReplicaIfNotPresent(storageInfo, ucBlock.reportedBlock, ucBlock.reportedState); if (ucBlock.reportedState == ReplicaState.FINALIZED && @@ -3945,6 +3958,20 @@ public static LocatedStripedBlock newLocatedStripedBlock( null); } + public LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, + DatanodeStorageInfo[] locs, long offset) throws IOException { + final LocatedBlock lb; + if (info.isStriped()) { + lb = newLocatedStripedBlock(eb, locs, + ((BlockInfoStripedUnderConstruction)info).getBlockIndices(), + offset, false); + } else { + lb = newLocatedBlock(eb, locs, offset, false); + } + setBlockToken(lb, BlockTokenIdentifier.AccessMode.WRITE); + return lb; + } + /** * This class is used internally by {@link this#computeRecoveryWorkForBlocks} * to represent a task to recover a block through replication or erasure 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 7f82d2d9686d5..3179b938452dc 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 @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hdfs.server.namenode; -import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; @@ -60,9 +59,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY; @@ -88,8 +87,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER; -import static org.apache.hadoop.util.Time.now; import static org.apache.hadoop.util.Time.monotonicNow; +import static org.apache.hadoop.util.Time.now; import java.io.BufferedWriter; import java.io.ByteArrayInputStream; @@ -140,6 +139,7 @@ import org.apache.hadoop.crypto.CryptoProtocolVersion; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; +import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion; import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries; import org.apache.hadoop.fs.CacheFlag; import org.apache.hadoop.fs.ContentSummary; @@ -152,6 +152,7 @@ import org.apache.hadoop.fs.InvalidPathException; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; @@ -160,10 +161,8 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; -import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.ServiceFailedException; -import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; @@ -172,6 +171,7 @@ import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CachePoolEntry; @@ -180,15 +180,15 @@ 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.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; -import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; 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.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; @@ -209,7 +209,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -217,7 +217,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; @@ -3799,7 +3798,8 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, } truncatedBlock = iFile.getLastBlock(); - long recoveryId = BlockInfo.getBlockRecoveryId(truncatedBlock); + final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)truncatedBlock; + final long recoveryId = uc.getBlockRecoveryId(); copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId(); if(recoveryId != newgenerationstamp) { throw new IOException("The recovery id " + newgenerationstamp From 6bacaa9a5233cbad7f311ccd9d8f8dc9375c732d Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 11 May 2015 12:22:12 -0700 Subject: [PATCH 106/212] HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8327 and HDFS-8357). Contributed by Zhe Zhang. --- .../server/blockmanagement/BlockInfo.java | 12 +++++- .../blockmanagement/BlockInfoContiguous.java | 38 ------------------- .../server/blockmanagement/BlockManager.java | 4 +- .../erasurecode/ErasureCodingWorker.java | 3 +- .../hdfs/server/namenode/INodeFile.java | 10 ++--- .../server/namenode/TestStripedINodeFile.java | 8 ++-- .../namenode/TestTruncateQuotaUpdate.java | 3 +- 7 files changed, 23 insertions(+), 55 deletions(-) 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 aebfbb185639f..61068b96dc768 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 @@ -88,13 +88,21 @@ DatanodeStorageInfo getStorageInfo(int index) { BlockInfo getPrevious(int index) { assert this.triplets != null : "BlockInfo is not initialized"; assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; - return (BlockInfo) triplets[index*3+1]; + BlockInfo info = (BlockInfo)triplets[index*3+1]; + assert info == null || + info.getClass().getName().startsWith(BlockInfo.class.getName()) : + "BlockInfo is expected at " + index*3; + return info; } BlockInfo getNext(int index) { assert this.triplets != null : "BlockInfo is not initialized"; assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; - return (BlockInfo) triplets[index*3+2]; + BlockInfo info = (BlockInfo)triplets[index*3+2]; + assert info == null || info.getClass().getName().startsWith( + BlockInfo.class.getName()) : + "BlockInfo is expected at " + index*3; + return info; } void setStorageInfo(int index, DatanodeStorageInfo storage) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index d3051a300ffb9..eeab076d5769d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -47,18 +47,6 @@ protected BlockInfoContiguous(BlockInfoContiguous from) { this.setBlockCollection(from.getBlockCollection()); } - public BlockCollection getBlockCollection() { - return bc; - } - - public void setBlockCollection(BlockCollection bc) { - this.bc = bc; - } - - public boolean isDeleted() { - return (bc == null); - } - public DatanodeDescriptor getDatanode(int index) { DatanodeStorageInfo storage = getStorageInfo(index); return storage == null ? null : storage.getDatanodeDescriptor(); @@ -70,32 +58,6 @@ DatanodeStorageInfo getStorageInfo(int index) { return (DatanodeStorageInfo)triplets[index*3]; } - private BlockInfoContiguous getPrevious(int index) { - assert this.triplets != null : "BlockInfo is not initialized"; - assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; - BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1]; - assert info == null || - info.getClass().getName().startsWith(BlockInfoContiguous.class.getName()) : - "BlockInfo is expected at " + index*3; - return info; - } - - BlockInfoContiguous getNext(int index) { - assert this.triplets != null : "BlockInfo is not initialized"; - assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; - BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2]; - assert info == null || info.getClass().getName().startsWith( - BlockInfoContiguous.class.getName()) : - "BlockInfo is expected at " + index*3; - return info; - } - - private void setStorageInfo(int index, DatanodeStorageInfo storage) { - assert this.triplets != null : "BlockInfo is not initialized"; - assert index >= 0 && index*3 < triplets.length : "Index is out of bound"; - triplets[index*3] = storage; - } - /** * Return the previous block on the block list for the datanode at * position index. Set the previous block on the list to "to". 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 aa0343a3a1b4f..5b876f951750e 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 @@ -2477,7 +2477,7 @@ private void processQueuedMessages(Iterable rbis) if (rbi.getReportedState() == null) { // This is a DELETE_BLOCK request DatanodeStorageInfo storageInfo = rbi.getStorageInfo(); - removeStoredBlock(rbi.getBlock(), + removeStoredBlock(getStoredBlock(rbi.getBlock()), storageInfo.getDatanodeDescriptor()); } else { processAndHandleReportedBlock(rbi.getStorageInfo(), @@ -3222,7 +3222,7 @@ private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block, QUEUE_REASON_FUTURE_GENSTAMP); return; } - removeStoredBlock(block, node); + removeStoredBlock(getStoredBlock(block), node); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index c4e568f2841c0..5ede508b63b83 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSPacket; +import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.RemoteBlockReader2; import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.net.TcpPeerServer; @@ -872,7 +873,7 @@ private int initTargetStreams(boolean[] targetsStatus) { unbufIn = saslStreams.in; out = new DataOutputStream(new BufferedOutputStream(unbufOut, - HdfsServerConstants.SMALL_BUFFER_SIZE)); + DFSUtil.getSmallBufferSize(conf))); in = new DataInputStream(unbufIn); DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId()); 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 f87a29c0e2e2a..cc187700e4b58 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 @@ -706,11 +706,7 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, */ public final QuotaCounts computeQuotaUsageWithStriped( BlockStoragePolicySuite bsps, QuotaCounts counts) { - long nsDelta = 1; - final long ssDelta = storagespaceConsumed(); - counts.addNameSpace(nsDelta); - counts.addStorageSpace(ssDelta); - return counts; + return null; } @Override @@ -979,11 +975,11 @@ void computeQuotaDeltaForTruncate( } 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(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 7a330b9c48a88..a725e6ba3da77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -109,8 +109,8 @@ public void testBlockStripedConsumedSpace() // a. * ( - 1) * = 0 // b. % = 1 // c. * = 1 * 3 - assertEquals(4, inf.storagespaceConsumedWithStriped()); - assertEquals(4, inf.storagespaceConsumed()); + assertEquals(4, inf.storagespaceConsumedWithStriped(null)); + assertEquals(4, inf.storagespaceConsumed(null)); } @Test @@ -134,8 +134,8 @@ public void testMultipleBlockStripedConsumedSpace() inf.addBlock(blockInfoStriped1); inf.addBlock(blockInfoStriped2); // This is the double size of one block in above case. - assertEquals(4 * 2, inf.storagespaceConsumedWithStriped()); - assertEquals(4 * 2, inf.storagespaceConsumed()); + assertEquals(4 * 2, inf.storagespaceConsumedWithStriped(null)); + assertEquals(4 * 2, inf.storagespaceConsumed(null)); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java index f6b18e6be17b1..57159dbec2c14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java @@ -20,6 +20,7 @@ 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList; @@ -77,7 +78,7 @@ public void testTruncateWithoutSnapshot() { @Test public void testTruncateWithSnapshotNoDivergence() { INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION); - addSnapshotFeature(file, file.getBlocks()); + addSnapshotFeature(file, file.getContiguousBlocks()); // case 4: truncate to 1.5 blocks // all the blocks are in snapshot. truncate need to allocate a new block From 8d3030f064116a657c2cbb7c7560af6bed1d5586 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 11 May 2015 21:10:23 -0700 Subject: [PATCH 107/212] HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread). Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 164 ++++-- .../erasurecode/ErasureCodingWorker.java | 10 +- .../hadoop/hdfs/util/StripedBlockUtil.java | 517 ++++++++++++++++-- .../hdfs/TestDFSStripedInputStream.java | 97 +++- .../hadoop/hdfs/TestWriteReadStripedFile.java | 49 ++ 6 files changed, 768 insertions(+), 72 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index c7d01c70c17ad..0acf746194f39 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -195,3 +195,6 @@ HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction. (Tsz Wo Nicholas Sze via jing9) + + HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread). + (Zhe Zhang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 7425e753441ae..7678fae0672ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -21,15 +21,27 @@ import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.StorageType; -import org.apache.hadoop.hdfs.protocol.*; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.ByteBufferPool; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; import org.apache.hadoop.io.erasurecode.ECSchema; + import org.apache.hadoop.net.NetUtils; import org.apache.htrace.Span; import org.apache.htrace.Trace; @@ -37,10 +49,12 @@ import java.io.EOFException; import java.io.IOException; +import java.io.InterruptedIOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.EnumSet; import java.util.Set; +import java.util.Collection; import java.util.Map; import java.util.HashMap; import java.util.concurrent.CompletionService; @@ -51,7 +65,6 @@ import java.util.concurrent.Callable; import java.util.concurrent.Future; - /****************************************************************************** * DFSStripedInputStream reads from striped block groups, illustrated below: * @@ -125,6 +138,7 @@ boolean include(long pos) { private final short parityBlkNum; /** the buffer for a complete stripe */ private ByteBuffer curStripeBuf; + private final ECSchema schema; /** * indicate the start/end offset of the current buffered stripe in the * block group @@ -137,6 +151,7 @@ boolean include(long pos) { super(dfsClient, src, verifyChecksum); assert schema != null; + this.schema = schema; cellSize = schema.getChunkSize(); dataBlkNum = (short) schema.getNumDataUnits(); parityBlkNum = (short) schema.getNumParityUnits(); @@ -472,12 +487,10 @@ private int copy(ReaderStrategy strategy, int offset, int length) { */ @Override protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { - LocatedBlock lb = super.getBlockAt(blkStartOffset); - assert lb instanceof LocatedStripedBlock : "NameNode should return a " + - "LocatedStripedBlock for a striped file"; + LocatedBlock lb = getBlockGroupAt(blkStartOffset); - int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize) - % dataBlkNum); + int idx = (int) ((blkStartOffset - lb.getStartOffset()) + % (dataBlkNum + parityBlkNum)); // If indexing information is returned, iterate through the index array // to find the entry for position idx in the group LocatedStripedBlock lsb = (LocatedStripedBlock) lb; @@ -509,48 +522,121 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { - Map, Integer> futures = new HashMap<>(); - CompletionService stripedReadsService = - new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); - int len = (int) (end - start + 1); - // Refresh the striped block group LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset); + AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, blockGroup, + start, end, buf, offset); + for (AlignedStripe stripe : stripes) { + fetchOneStripe(blockGroup, buf, stripe, corruptedBlockMap); + } + } - // Planning the portion of I/O for each shard - ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start, - len, offset); - + private void fetchOneStripe(LocatedStripedBlock blockGroup, + byte[] buf, AlignedStripe alignedStripe, Map> corruptedBlockMap) throws IOException { + Map, Integer> futures = new HashMap<>(); + CompletionService service = + new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); + if (alignedStripe.getSpanInBlock() == 0) { + DFSClient.LOG.warn("Trying to read an empty stripe from" + blockGroup); + return; + } // Parse group to get chosen DN location LocatedBlock[] blks = StripedBlockUtil. parseStripedBlockGroup(blockGroup, cellSize, dataBlkNum, parityBlkNum); - for (short i = 0; i < dataBlkNum; i++) { - ReadPortion rp = readPortions[i]; - if (rp.getReadLength() <= 0) { - continue; + if (alignedStripe.chunks[i] != null + && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { + fetchOneStripingChunk(futures, service, blks[i], alignedStripe, i, + corruptedBlockMap); } - DatanodeInfo loc = blks[i].getLocations()[0]; - StorageType type = blks[i].getStorageTypes()[0]; - DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( - loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), - type); - Callable readCallable = getFromOneDataNode(dnAddr, - blks[i].getStartOffset(), rp.getStartOffsetInBlock(), - rp.getStartOffsetInBlock() + rp.getReadLength() - 1, buf, - rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i); - Future getFromDNRequest = stripedReadsService.submit(readCallable); - DFSClient.LOG.debug("Submitting striped read request for " + blks[i]); - futures.put(getFromDNRequest, (int) i); } + // Input buffers for potential decode operation, which remains null until + // first read failure + byte[][] decodeInputs = null; while (!futures.isEmpty()) { try { - waitNextCompletion(stripedReadsService, futures); + StripingChunkReadResult r = getNextCompletedStripedRead( + service, futures, 0); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Read task returned: " + r + ", for stripe " + alignedStripe); + } + StripingChunk returnedChunk = alignedStripe.chunks[r.index]; + Preconditions.checkNotNull(returnedChunk); + Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING); + if (r.state == StripingChunkReadResult.SUCCESSFUL) { + returnedChunk.state = StripingChunk.FETCHED; + alignedStripe.fetchedChunksNum++; + if (alignedStripe.fetchedChunksNum == dataBlkNum) { + clearFutures(futures.keySet()); + break; + } + } else { + returnedChunk.state = StripingChunk.MISSING; + alignedStripe.missingChunksNum++; + if (alignedStripe.missingChunksNum > parityBlkNum) { + clearFutures(futures.keySet()); + throw new IOException("Too many blocks are missing: " + alignedStripe); + } + // When seeing first missing block, initialize decode input buffers + if (decodeInputs == null) { + decodeInputs = initDecodeInputs(alignedStripe, dataBlkNum, parityBlkNum); + } + for (int i = 0; i < alignedStripe.chunks.length; i++) { + StripingChunk chunk = alignedStripe.chunks[i]; + Preconditions.checkNotNull(chunk); + if (chunk.state == StripingChunk.REQUESTED && i <= dataBlkNum) { + fetchOneStripingChunk(futures, service, blks[i], alignedStripe, i, + corruptedBlockMap); + } + } + } } catch (InterruptedException ie) { - // Ignore and retry + String err = "Read request interrupted"; + DFSClient.LOG.error(err); + clearFutures(futures.keySet()); + // Don't decode if read interrupted + throw new InterruptedIOException(err); } } + + if (alignedStripe.missingChunksNum > 0) { + decodeAndFillBuffer(decodeInputs, buf, alignedStripe, + dataBlkNum, parityBlkNum); + } + } + + /** + * Schedule a single read request to an internal block + * @param block The internal block + * @param index Index of the internal block in the group + * @param corruptedBlockMap Map of corrupted blocks + */ + private void fetchOneStripingChunk(Map, Integer> futures, + final CompletionService service, final LocatedBlock block, + final AlignedStripe alignedStripe, final int index, + Map> corruptedBlockMap) { + DatanodeInfo loc = block.getLocations()[0]; + StorageType type = block.getStorageTypes()[0]; + DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( + loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), + type); + StripingChunk chunk = alignedStripe.chunks[index]; + chunk.state = StripingChunk.PENDING; + Callable readCallable = getFromOneDataNode(dnAddr, + block.getStartOffset(), alignedStripe.getOffsetInBlock(), + alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, chunk.buf, + chunk.getOffsets(), chunk.getLengths(), + corruptedBlockMap, index); + Future getFromDNRequest = service.submit(readCallable); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Submitting striped read request for " + index + + ". Info of the block: " + block + ", offset in block is " + + alignedStripe.getOffsetInBlock() + ", end is " + + (alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1)); + } + futures.put(getFromDNRequest, index); } private Callable getFromOneDataNode(final DNAddrPair datanode, @@ -609,4 +695,12 @@ public synchronized void releaseBuffer(ByteBuffer buffer) { throw new UnsupportedOperationException( "Not support enhanced byte buffer access."); } + + /** A variation to {@link DFSInputStream#cancelAll} */ + private void clearFutures(Collection> futures) { + for (Future future : futures) { + future.cancel(false); + } + futures.clear(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 5ede508b63b83..eedb1914ceb53 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -67,7 +67,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripedReadResult; +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; @@ -462,10 +462,10 @@ private int readMinimumStripedData4Recovery(int[] success) { int nsuccess = 0; while (!futures.isEmpty()) { try { - StripedReadResult result = + StripingChunkReadResult result = StripedBlockUtil.getNextCompletedStripedRead( readService, futures, STRIPED_READ_THRESHOLD_MILLIS); - if (result.state == StripedReadResult.SUCCESSFUL) { + if (result.state == StripingChunkReadResult.SUCCESSFUL) { success[nsuccess++] = result.index; if (nsuccess >= dataBlkNum) { // cancel remaining reads if we read successfully from minimum @@ -474,14 +474,14 @@ private int readMinimumStripedData4Recovery(int[] success) { futures.clear(); break; } - } else if (result.state == StripedReadResult.FAILED) { + } else if (result.state == StripingChunkReadResult.FAILED) { // If read failed for some source, we should not use it anymore // and schedule read from a new source. StripedReader failedReader = stripedReaders.get(result.index); closeBlockReader(failedReader.blockReader); failedReader.blockReader = null; scheduleNewRead(used); - } else if (result.state == StripedReadResult.TIMEOUT) { + } else if (result.state == StripingChunkReadResult.TIMEOUT) { // If timeout, we also schedule a new read. scheduleNewRead(used); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 45bbf6bf6d4c6..f7ae88a47372d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -22,16 +22,18 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSClient; +import org.apache.hadoop.hdfs.DFSStripedOutputStream; 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.protocol.LocatedStripedBlock; import com.google.common.base.Preconditions; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; @@ -85,7 +87,7 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, - bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), + bg.getStartOffset() + idxInBlockGroup, bg.isCorrupt(), null); } @@ -238,33 +240,37 @@ public static ReadPortion[] planReadPortions(final int dataBlkNum, /** * Get the next completed striped read task * - * @return {@link StripedReadResult} indicating the status of the read task + * @return {@link StripingChunkReadResult} indicating the status of the read task * succeeded, and the block index of the task. If the method times * out without getting any completed read tasks, -1 is returned as * block index. * @throws InterruptedException */ - public static StripedReadResult getNextCompletedStripedRead( + public static StripingChunkReadResult getNextCompletedStripedRead( CompletionService readService, Map, Integer> futures, final long threshold) throws InterruptedException { Preconditions.checkArgument(!futures.isEmpty()); - Preconditions.checkArgument(threshold > 0); Future future = null; try { - future = readService.poll(threshold, TimeUnit.MILLISECONDS); + if (threshold > 0) { + future = readService.poll(threshold, TimeUnit.MILLISECONDS); + } else { + future = readService.take(); + } if (future != null) { future.get(); - return new StripedReadResult(futures.remove(future), - StripedReadResult.SUCCESSFUL); + return new StripingChunkReadResult(futures.remove(future), + StripingChunkReadResult.SUCCESSFUL); } else { - return new StripedReadResult(StripedReadResult.TIMEOUT); + return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT); } } catch (ExecutionException e) { - return new StripedReadResult(futures.remove(future), - StripedReadResult.FAILED); + DFSClient.LOG.error("ExecutionException " + e); + return new StripingChunkReadResult(futures.remove(future), + StripingChunkReadResult.FAILED); } catch (CancellationException e) { - return new StripedReadResult(futures.remove(future), - StripedReadResult.CANCELLED); + return new StripingChunkReadResult(futures.remove(future), + StripingChunkReadResult.CANCELLED); } } @@ -291,26 +297,247 @@ public static long spaceConsumedByStripedBlock(long numDataBlkBytes, } /** - * This class represents the portion of I/O associated with each block in the - * striped block group. + * Initialize the decoding input buffers based on the chunk states in an + * AlignedStripe */ - public static class ReadPortion { + public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, + int dataBlkNum, int parityBlkNum) { + byte[][] decodeInputs = + new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()]; + for (int i = 0; i < alignedStripe.chunks.length; i++) { + StripingChunk chunk = alignedStripe.chunks[i]; + if (chunk == null) { + alignedStripe.chunks[i] = new StripingChunk(decodeInputs[i]); + alignedStripe.chunks[i].offsetsInBuf.add(0); + alignedStripe.chunks[i].lengthsInBuf.add((int) alignedStripe.getSpanInBlock()); + } else if (chunk.state == StripingChunk.FETCHED) { + int posInBuf = 0; + for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { + System.arraycopy(chunk.buf, chunk.offsetsInBuf.get(j), + decodeInputs[i], posInBuf, chunk.lengthsInBuf.get(j)); + posInBuf += chunk.lengthsInBuf.get(j); + } + } else if (chunk.state == StripingChunk.ALLZERO) { + Arrays.fill(decodeInputs[i], (byte)0); + } + } + return decodeInputs; + } + + /** + * Decode based on the given input buffers and schema + */ + public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf, + AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum) { + int[] decodeIndices = new int[parityBlkNum]; + int pos = 0; + for (int i = 0; i < alignedStripe.chunks.length; i++) { + if (alignedStripe.chunks[i].state != StripingChunk.FETCHED && + alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { + decodeIndices[pos++] = i; + } + } + + byte[][] outputs = new byte[parityBlkNum][(int) alignedStripe.getSpanInBlock()]; + RSRawDecoder rsRawDecoder = new RSRawDecoder(); + rsRawDecoder.initialize(dataBlkNum, parityBlkNum, (int) alignedStripe.getSpanInBlock()); + rsRawDecoder.decode(decodeInputs, decodeIndices, outputs); + + for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { + StripingChunk chunk = alignedStripe.chunks[i]; + if (chunk.state == StripingChunk.MISSING) { + int srcPos = 0; + for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { + //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 +// System.arraycopy(outputs[i], srcPos, buf, chunk.offsetsInBuf.get(j), +// chunk.lengthsInBuf.get(j)); + Arrays.fill(buf, chunk.offsetsInBuf.get(j), + chunk.offsetsInBuf.get(j) + chunk.lengthsInBuf.get(j), (byte)7); + srcPos += chunk.lengthsInBuf.get(j); + } + } + } + } + + /** + * This method divides a requested byte range into an array of + * {@link AlignedStripe} + * + * + * At most 5 stripes will be generated from each logical range + * TODO: cleanup and get rid of planReadPortions + */ + public static AlignedStripe[] divideByteRangeIntoStripes ( + ECSchema ecSchema, LocatedStripedBlock blockGroup, long start, long end, + byte[] buf, int offsetInBuf) { + // TODO: change ECSchema naming to use cell size instead of chunk size + + // Step 0: analyze range and calculate basic parameters + int cellSize = ecSchema.getChunkSize(); + int dataBlkNum = ecSchema.getNumDataUnits(); + int len = (int) (end - start + 1); + int firstCellIdxInBG = (int) (start / cellSize); + int lastCellIdxInBG = (int) (end / cellSize); + int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len); + long firstCellOffsetInBlk = start % cellSize; + int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ? + firstCellSize : (int) (end % cellSize) + 1; + + // Step 1: get the unmerged ranges on each internal block + // TODO: StripingCell should carry info on size and start offset (HDFS-8320) + VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, + firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk, + lastCellSize); + + // Step 2: merge into at most 5 stripes + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); + + // Step 3: calculate each chunk's position in destination buffer + calcualteChunkPositionsInBuf(ecSchema, blockGroup, buf, offsetInBuf, + firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk, + lastCellSize, stripes); + + // Step 4: prepare ALLZERO blocks + prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum); + + return stripes; + } + + private static VerticalRange[] getRangesForInternalBlocks (ECSchema ecSchema, + int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize, + long firstCellOffsetInBlk, int lastCellSize) { + int cellSize = ecSchema.getChunkSize(); + int dataBlkNum = ecSchema.getNumDataUnits(); + + StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG); + StripingCell lastCell = new StripingCell(ecSchema, lastCellIdxInBG); + + VerticalRange ranges[] = new VerticalRange[dataBlkNum]; + ranges[firstCell.idxInStripe] = + new VerticalRange(firstCellOffsetInBlk, firstCellSize); + for (int i = firstCellIdxInBG + 1; i < lastCellIdxInBG; i++) { + // iterate through all cells and update the list of StripeRanges + StripingCell cell = new StripingCell(ecSchema, i); + if (ranges[cell.idxInStripe] == null) { + ranges[cell.idxInStripe] = new VerticalRange( + cell.idxInInternalBlk * cellSize, cellSize); + } else { + ranges[cell.idxInStripe].spanInBlock += cellSize; + } + } + if (ranges[lastCell.idxInStripe] == null) { + ranges[lastCell.idxInStripe] = new VerticalRange( + lastCell.idxInInternalBlk * cellSize, lastCellSize); + } else if (lastCell.idxInBlkGroup != firstCell.idxInBlkGroup) { + ranges[lastCell.idxInStripe].spanInBlock += lastCellSize; + } + + return ranges; + } + + private static AlignedStripe[] mergeRangesForInternalBlocks(ECSchema ecSchema, + VerticalRange[] ranges) { + int dataBlkNum = ecSchema.getNumDataUnits(); + int parityBlkNum = ecSchema.getNumParityUnits(); + List stripes = new ArrayList<>(); + SortedSet stripePoints = new TreeSet<>(); + for (VerticalRange r : ranges) { + if (r != null) { + stripePoints.add(r.offsetInBlock); + stripePoints.add(r.offsetInBlock + r.spanInBlock); + } + } + + long prev = -1; + for (long point : stripePoints) { + if (prev >= 0) { + stripes.add(new AlignedStripe(prev, point - prev, + dataBlkNum + parityBlkNum)); + } + prev = point; + } + return stripes.toArray(new AlignedStripe[stripes.size()]); + } + + private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, + LocatedStripedBlock blockGroup, byte[] buf, int offsetInBuf, + int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize, + long firstCellOffsetInBlk, int lastCellSize, AlignedStripe[] stripes) { + int cellSize = ecSchema.getChunkSize(); + int dataBlkNum = ecSchema.getNumDataUnits(); + // Step 3: calculate each chunk's position in destination buffer /** - * startOffsetInBlock - * | - * v - * |<-lengths[0]->|<- lengths[1] ->|<-lengths[2]->| + * | <--------------- AlignedStripe --------------->| + * + * |<- length_0 ->|<-- length_1 -->|<- length_2 ->| * +------------------+------------------+----------------+ - * | cell_0 | cell_3 | cell_6 | <- blk_0 + * | cell_0_0_0 | cell_3_1_0 | cell_6_2_0 | <- blk_0 * +------------------+------------------+----------------+ * _/ \_______________________ * | | - * v offsetsInBuf[0] v offsetsInBuf[1] - * +------------------------------------------------------+ - * | cell_0 | cell_1 and cell_2 |cell_3 ...| <- buf - * | (partial) | (from blk_1 and blk_2) | | - * +------------------------------------------------------+ + * v offset_0 v offset_1 + * +----------------------------------------------------------+ + * | cell_0_0_0 | cell_1_0_1 and cell_2_0_2 |cell_3_1_0 ...| <- buf + * | (partial) | (from blk_1 and blk_2) | | + * +----------------------------------------------------------+ + * + * Cell indexing convention defined in {@link StripingCell} */ + int done = 0; + for (int i = firstCellIdxInBG; i <= lastCellIdxInBG; i++) { + StripingCell cell = new StripingCell(ecSchema, i); + long cellStart = i == firstCellIdxInBG ? + firstCellOffsetInBlk : cell.idxInInternalBlk * cellSize; + int cellLen; + if (i == firstCellIdxInBG) { + cellLen = firstCellSize; + } else if (i == lastCellIdxInBG) { + cellLen = lastCellSize; + } else { + cellLen = cellSize; + } + long cellEnd = cellStart + cellLen - 1; + for (AlignedStripe s : stripes) { + long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1; + long overlapStart = Math.max(cellStart, s.getOffsetInBlock()); + long overlapEnd = Math.min(cellEnd, stripeEnd); + int overLapLen = (int) (overlapEnd - overlapStart + 1); + if (overLapLen <= 0) { + continue; + } + if (s.chunks[cell.idxInStripe] == null) { + s.chunks[cell.idxInStripe] = new StripingChunk(buf); + } + + s.chunks[cell.idxInStripe].offsetsInBuf. + add((int)(offsetInBuf + done + overlapStart - cellStart)); + s.chunks[cell.idxInStripe].lengthsInBuf.add(overLapLen); + } + done += cellLen; + } + } + + private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, + byte[] buf, AlignedStripe[] stripes, int cellSize, int dataBlkNum) { + for (AlignedStripe s : stripes) { + for (int i = 0; i < dataBlkNum; i++) { + long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(), + cellSize, dataBlkNum, i); + if (internalBlkLen <= s.getOffsetInBlock()) { + Preconditions.checkState(s.chunks[i] == null); + s.chunks[i] = new StripingChunk(buf); + s.chunks[i].state = StripingChunk.ALLZERO; + } + } + } + } + + /** + * This class represents the portion of I/O associated with each block in the + * striped block group. + * TODO: consolidate ReadPortion with AlignedStripe + */ + public static class ReadPortion { private long startOffsetInBlock = 0; private int readLength = 0; public final List offsetsInBuf = new ArrayList<>(); @@ -349,12 +576,235 @@ void addReadLength(int extraLength) { } } + /** + * The unit of encoding used in {@link DFSStripedOutputStream} + * | <------- Striped Block Group -------> | + * blk_0 blk_1 blk_2 + * | | | + * v v v + * +----------+ +----------+ +----------+ + * |cell_0_0_0| |cell_1_0_1| |cell_2_0_2| + * +----------+ +----------+ +----------+ + * |cell_3_1_0| |cell_4_1_1| |cell_5_1_2| <- {@link idxInBlkGroup} = 5 + * +----------+ +----------+ +----------+ {@link idxInInternalBlk} = 1 + * {@link idxInStripe} = 2 + * A StripingCell is a special instance of {@link StripingChunk} whose offset + * and size align with the cell used when writing data. + * TODO: consider parity cells + */ + public static class StripingCell { + public final ECSchema schema; + /** Logical order in a block group, used when doing I/O to a block group */ + public final int idxInBlkGroup; + public final int idxInInternalBlk; + public final int idxInStripe; + + public StripingCell(ECSchema ecSchema, int idxInBlkGroup) { + this.schema = ecSchema; + this.idxInBlkGroup = idxInBlkGroup; + this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits(); + this.idxInStripe = idxInBlkGroup - + this.idxInInternalBlk * ecSchema.getNumDataUnits(); + } + + public StripingCell(ECSchema ecSchema, int idxInInternalBlk, + int idxInStripe) { + this.schema = ecSchema; + this.idxInInternalBlk = idxInInternalBlk; + this.idxInStripe = idxInStripe; + this.idxInBlkGroup = + idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe; + } + } + + /** + * Given a requested byte range on a striped block group, an AlignedStripe + * represents a {@link VerticalRange} that is aligned with both the byte range + * and boundaries of all internal blocks. As illustrated in the diagram, any + * given byte range on a block group leads to 1~5 AlignedStripe's. + * + * |<-------- Striped Block Group -------->| + * blk_0 blk_1 blk_2 blk_3 blk_4 + * +----+ | +----+ +----+ + * |full| | | | | | <- AlignedStripe0: + * +----+ |~~~~| | |~~~~| |~~~~| 1st cell is partial + * |part| | | | | | | | <- AlignedStripe1: byte range + * +----+ +----+ +----+ | |~~~~| |~~~~| doesn't start at 1st block + * |full| |full| |full| | | | | | + * |cell| |cell| |cell| | | | | | <- AlignedStripe2 (full stripe) + * | | | | | | | | | | | + * +----+ +----+ +----+ | |~~~~| |~~~~| + * |full| |part| | | | | | <- AlignedStripe3: byte range + * |~~~~| +----+ | |~~~~| |~~~~| doesn't end at last block + * | | | | | | | <- AlignedStripe4: + * +----+ | +----+ +----+ last cell is partial + * | + * <---- data blocks ----> | <--- parity ---> + * + * An AlignedStripe is the basic unit of reading from a striped block group, + * because within the AlignedStripe, all internal blocks can be processed in + * a uniform manner. + * + * The coverage of an AlignedStripe on an internal block is represented as a + * {@link StripingChunk}. + * To simplify the logic of reading a logical byte range from a block group, + * a StripingChunk is either completely in the requested byte range or + * completely outside the requested byte range. + */ + public static class AlignedStripe { + public VerticalRange range; + /** status of each chunk in the stripe */ + public final StripingChunk[] chunks; + public int fetchedChunksNum = 0; + public int missingChunksNum = 0; + + public AlignedStripe(long offsetInBlock, long length, int width) { + Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0); + this.range = new VerticalRange(offsetInBlock, length); + this.chunks = new StripingChunk[width]; + } + + public AlignedStripe(VerticalRange range, int width) { + this.range = range; + this.chunks = new StripingChunk[width]; + } + + public boolean include(long pos) { + return range.include(pos); + } + + public long getOffsetInBlock() { + return range.offsetInBlock; + } + + public long getSpanInBlock() { + return range.spanInBlock; + } + + @Override + public String toString() { + return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock + + ", fetchedChunksNum=" + fetchedChunksNum + + ", missingChunksNum=" + missingChunksNum; + } + } + + /** + * A simple utility class representing an arbitrary vertical inclusive range + * starting at {@link offsetInBlock} and lasting for {@link length} bytes in + * an internal block. Note that VerticalRange doesn't necessarily align with + * {@link StripingCell}. + * + * |<- Striped Block Group ->| + * blk_0 + * | + * v + * +-----+ + * |~~~~~| <-- {@link offsetInBlock} + * | | ^ + * | | | + * | | | {@link spanInBlock} + * | | v + * |~~~~~| --- + * | | + * +-----+ + */ + public static class VerticalRange { + /** start offset in the block group (inclusive) */ + public long offsetInBlock; + /** length of the stripe range */ + public long spanInBlock; + + public VerticalRange(long offsetInBlock, long length) { + Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0); + this.offsetInBlock = offsetInBlock; + this.spanInBlock = length; + } + + /** whether a position is in the range */ + public boolean include(long pos) { + return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock; + } + } + + /** + * Indicates the coverage of an {@link AlignedStripe} on an internal block, + * and the state of the chunk in the context of the read request. + * + * |<---------------- Striped Block Group --------------->| + * blk_0 blk_1 blk_2 blk_3 blk_4 + * +---------+ | +----+ +----+ + * null null |REQUESTED| | |null| |null| <- AlignedStripe0 + * +---------+ |---------| | |----| |----| + * null |REQUESTED| |REQUESTED| | |null| |null| <- AlignedStripe1 + * +---------+ +---------+ +---------+ | +----+ +----+ + * |REQUESTED| |REQUESTED| ALLZERO | |null| |null| <- AlignedStripe2 + * +---------+ +---------+ | +----+ +----+ + * <----------- data blocks ------------> | <--- parity ---> + * + * The class also carries {@link buf}, {@link offsetsInBuf}, and + * {@link lengthsInBuf} to define how read task for this chunk should deliver + * the returned data. + */ + public static class StripingChunk { + /** Chunk has been successfully fetched */ + public static final int FETCHED = 0x01; + /** Chunk has encountered failed when being fetched */ + public static final int MISSING = 0x02; + /** Chunk being fetched (fetching task is in-flight) */ + public static final int PENDING = 0x04; + /** + * Chunk is requested either by application or for decoding, need to + * schedule read task + */ + public static final int REQUESTED = 0X08; + /** + * Internal block is short and has no overlap with chunk. Chunk considered + * all-zero bytes in codec calculations. + */ + public static final int ALLZERO = 0X0f; + + /** + * If a chunk is completely in requested range, the state transition is: + * REQUESTED (when AlignedStripe created) -> PENDING -> {FETCHED | MISSING} + * If a chunk is completely outside requested range (including parity + * chunks), state transition is: + * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ... + */ + public int state = REQUESTED; + public byte[] buf; + public List offsetsInBuf; + public List lengthsInBuf; + + public StripingChunk(byte[] buf) { + this.buf = buf; + this.offsetsInBuf = new ArrayList<>(); + this.lengthsInBuf = new ArrayList<>(); + } + + public int[] getOffsets() { + int[] offsets = new int[offsetsInBuf.size()]; + for (int i = 0; i < offsets.length; i++) { + offsets[i] = offsetsInBuf.get(i); + } + return offsets; + } + + public int[] getLengths() { + int[] lens = new int[this.lengthsInBuf.size()]; + for (int i = 0; i < lens.length; i++) { + lens[i] = this.lengthsInBuf.get(i); + } + return lens; + } + } + /** * This class represents result from a striped read request. * If the task was successful or the internal computation failed, * an index is also returned. */ - public static class StripedReadResult { + public static class StripingChunkReadResult { public static final int SUCCESSFUL = 0x01; public static final int FAILED = 0x02; public static final int TIMEOUT = 0x04; @@ -363,18 +813,23 @@ public static class StripedReadResult { public final int index; public final int state; - public StripedReadResult(int state) { + public StripingChunkReadResult(int state) { Preconditions.checkArgument(state == TIMEOUT, "Only timeout result should return negative index."); this.index = -1; this.state = state; } - public StripedReadResult(int index, int state) { + public StripingChunkReadResult(int index, int state) { Preconditions.checkArgument(state != TIMEOUT, "Timeout result should return negative index."); this.index = index; this.state = state; } + + @Override + public String toString() { + return "(index=" + index + ", state =" + state + ")"; + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 3f79933a32961..452cc2b9e81e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -133,8 +134,102 @@ public void testPread() throws Exception { byte[] readBuffer = new byte[readSize]; int ret = in.read(0, readBuffer, 0, readSize); + byte[] expected = new byte[readSize]; + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[posInFile] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } + } + } + assertEquals(readSize, ret); - // TODO: verify read results with patterned data from HDFS-8117 + assertArrayEquals(expected, readBuffer); + } + + @Test + public void testPreadWithDNFailure() throws Exception { + final int numBlocks = 4; + final int failedDNIdx = 2; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCK_GROUP_SIZE); + + assert lbs.get(0) instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); + for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); + } + DFSStripedInputStream in = + new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, + ErasureCodingSchemaManager.getSystemDefaultSchema()); + int readSize = BLOCK_GROUP_SIZE; + byte[] readBuffer = new byte[readSize]; + byte[] expected = new byte[readSize]; + cluster.stopDataNode(failedDNIdx); + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[posInFile] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } + } + } + + // Update the expected content for decoded data + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + byte[][] decodeInputs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][CELLSIZE]; + int[] missingBlkIdx = new int[]{failedDNIdx, DATA_BLK_NUM+1, DATA_BLK_NUM+2}; + byte[][] decodeOutputs = new byte[PARITY_BLK_NUM][CELLSIZE]; + for (int j = 0; j < DATA_BLK_NUM; j++) { + int posInBuf = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE; + if (j != failedDNIdx) { + System.arraycopy(expected, posInBuf, decodeInputs[j], 0, CELLSIZE); + } + } + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + decodeInputs[DATA_BLK_NUM][k] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk); + } +// RSRawDecoder rsRawDecoder = new RSRawDecoder(); +// rsRawDecoder.initialize(DATA_BLK_NUM, PARITY_BLK_NUM, CELLSIZE); +// rsRawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs); + int posInBuf = i * CELLSIZE * DATA_BLK_NUM + failedDNIdx * CELLSIZE; +// System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE); + //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 + Arrays.fill(expected, posInBuf, posInBuf + CELLSIZE, (byte)7); + } + int delta = 10; + int done = 0; + // read a small delta, shouldn't trigger decode + // |cell_0 | + // |10 | + done += in.read(0, readBuffer, 0, delta); + assertEquals(delta, done); + // both head and trail cells are partial + // |c_0 |c_1 |c_2 |c_3 |c_4 |c_5 | + // |256K - 10|missing|256K|256K|256K - 10|not in range| + done += in.read(delta, readBuffer, delta, + CELLSIZE * (DATA_BLK_NUM - 1) - 2 * delta); + assertEquals(CELLSIZE * (DATA_BLK_NUM - 1) - delta, done); + // read the rest + done += in.read(done, readBuffer, done, readSize - done); + assertEquals(readSize, done); + assertArrayEquals(expected, readBuffer); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 5c6f449f5f75e..57d6eb9a0dd8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -18,10 +18,13 @@ package org.apache.hadoop.hdfs; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -321,4 +324,50 @@ private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) Assert.assertArrayEquals(bytes, result.array()); } } + + @Test + public void testWritePreadWithDNFailure() throws IOException { + final int failedDNIdx = 2; + final int length = cellSize * (dataBlocks + 2); + Path testPath = new Path("/foo"); + final byte[] bytes = generateBytes(length); + DFSTestUtil.writeFile(fs, testPath, new String(bytes)); + + // shut down the DN that holds the last internal data block + BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, + cellSize); + String name = (locs[0].getNames())[failedDNIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + break; + } + } + + // pread + int startOffsetInFile = cellSize * 5; + try (FSDataInputStream fsdis = fs.open(testPath)) { + byte[] buf = new byte[length]; + int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length); + Assert.assertEquals("The length of file should be the same to write size", + length - startOffsetInFile, readLen); + + RSRawDecoder rsRawDecoder = new RSRawDecoder(); + rsRawDecoder.initialize(dataBlocks, parityBlocks, 1); + byte[] expected = new byte[readLen]; + for (int i = startOffsetInFile; i < length; i++) { + //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 + if ((i / cellSize) % dataBlocks == failedDNIdx) { + expected[i - startOffsetInFile] = (byte)7; + } else { + expected[i - startOffsetInFile] = getByte(i); + } + } + for (int i = startOffsetInFile; i < length; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", + expected[i - startOffsetInFile], buf[i - startOffsetInFile]); + } + } + } } From 97a2396af685838c9fcb31e48573e758c124d8d7 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 12 May 2015 11:43:04 -0700 Subject: [PATCH 108/212] HDFS-8372. Erasure coding: compute storage type quotas for striped files, to be consistent with HDFS-8327. Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../FileWithStripedBlocksFeature.java | 12 +++-- .../hdfs/server/namenode/INodeFile.java | 53 ++++++++++++++----- .../server/namenode/TestStripedINodeFile.java | 22 ++++---- 4 files changed, 64 insertions(+), 26 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0acf746194f39..f026a5cde76c3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -198,3 +198,6 @@ HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread). (Zhe Zhang) + + HDFS-8372. Erasure coding: compute storage type quotas for striped files, + to be consistent with HDFS-8327. (Zhe Zhang via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java index 47445be5b6b03..94ab527e40c18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; /** * Feature for file with striped blocks @@ -78,20 +79,23 @@ void addBlock(BlockInfoStriped newBlock) { } } - boolean removeLastBlock(Block oldblock) { + BlockInfoStripedUnderConstruction removeLastBlock( + Block oldblock) { if (blocks == null || blocks.length == 0) { - return false; + return null; } int newSize = blocks.length - 1; if (!blocks[newSize].equals(oldblock)) { - return false; + return null; } + BlockInfoStripedUnderConstruction uc = + (BlockInfoStripedUnderConstruction) blocks[newSize]; //copy to a new list BlockInfoStriped[] newlist = new BlockInfoStriped[newSize]; System.arraycopy(blocks, 0, newlist, 0, newSize); setBlocks(newlist); - return true; + return uc; } void truncateStripedBlocks(int n) { 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 cc187700e4b58..154198c5206d7 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 @@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff; @@ -295,7 +296,7 @@ public void convertLastBlockToUC(BlockInfo lastBlock, * Remove a block from the block list. This block should be * the last one on the list. */ - BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) { + BlockInfoUnderConstruction removeLastBlock(Block oldblock) { Preconditions.checkState(isUnderConstruction(), "file is no longer under construction"); FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); @@ -317,7 +318,7 @@ BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) { return uc; } else { assert hasNoContiguousBlock(); - return null; + return sb.removeLastBlock(oldblock); } } @@ -676,7 +677,7 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, final long ssDeltaNoReplication; short replication; if (isStriped()) { - return computeQuotaUsageWithStriped(bsps, counts); + return computeQuotaUsageWithStriped(bsp, counts); } if (last < lastSnapshotId) { @@ -702,11 +703,15 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, } /** - * Compute quota of striped file + * Compute quota of striped file. Note that currently EC files do not support + * append/hflush/hsync, thus the file length recorded in snapshots should be + * the same with the current file length. */ public final QuotaCounts computeQuotaUsageWithStriped( - BlockStoragePolicySuite bsps, QuotaCounts counts) { - return null; + BlockStoragePolicy bsp, QuotaCounts counts) { + counts.addNameSpace(1); + counts.add(storagespaceConsumed(bsp)); + return counts; } @Override @@ -828,21 +833,44 @@ public final long computeFileSize(boolean includesLastUcBlock, * Use preferred block size for the last block if it is under construction. */ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { - QuotaCounts counts = new QuotaCounts.Builder().build(); if (isStriped()) { - return storagespaceConsumedWithStriped(bsp); + return storagespaceConsumedWithStriped(); } else { return storagespaceConsumedWithReplication(bsp); } } - public final QuotaCounts storagespaceConsumedWithStriped( - BlockStoragePolicy bsp) { - return null; + // TODO: support EC with heterogeneous storage + public final QuotaCounts storagespaceConsumedWithStriped() { + QuotaCounts counts = new QuotaCounts.Builder().build(); + BlockInfo[] blockInfos = getBlocks(); + if (blockInfos == null || blockInfos.length == 0) { + return counts; + } + + long size; + final int last = blockInfos.length - 1; + if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) { + BlockInfoStripedUnderConstruction blockInfoStripedUC + =(BlockInfoStripedUnderConstruction)blockInfos[last]; + size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum(); + } else { + // In case of last block is complete + BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last]; + size = blockInfoStriped.spaceConsumed(); + } + for (int i = 0; i < last; i++) { + BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i]; + size += blockInfoStriped.spaceConsumed(); + } + + counts.addStorageSpace(size); + return counts; } public final QuotaCounts storagespaceConsumedWithReplication( - BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build(); + BlockStoragePolicy bsp) { + QuotaCounts counts = new QuotaCounts.Builder().build(); final Iterable blocks; FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); if (sf == null) { @@ -965,6 +993,7 @@ public long collectBlocksBeyondMax(final long max, /** * compute the quota usage change for a truncate op * @param newLength the length for truncation + * TODO: properly handle striped blocks (HDFS-7622) **/ void computeQuotaDeltaForTruncate( long newLength, BlockStoragePolicy bsps, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index a725e6ba3da77..e24bc54a6c40b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -27,6 +27,7 @@ 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.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; @@ -45,6 +46,11 @@ public class TestStripedINodeFile { private static final PermissionStatus perm = new PermissionStatus( "userName", null, FsPermission.getDefault()); + private final BlockStoragePolicySuite defaultSuite = + BlockStoragePolicySuite.createDefaultSuite(); + private final BlockStoragePolicy defaultPolicy = + defaultSuite.getDefaultPolicy(); + private static INodeFile createStripedINodeFile() { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID); @@ -109,8 +115,8 @@ public void testBlockStripedConsumedSpace() // a. * ( - 1) * = 0 // b. % = 1 // c. * = 1 * 3 - assertEquals(4, inf.storagespaceConsumedWithStriped(null)); - assertEquals(4, inf.storagespaceConsumed(null)); + assertEquals(4, inf.storagespaceConsumedWithStriped().getStorageSpace()); + assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace()); } @Test @@ -134,8 +140,8 @@ public void testMultipleBlockStripedConsumedSpace() inf.addBlock(blockInfoStriped1); inf.addBlock(blockInfoStriped2); // This is the double size of one block in above case. - assertEquals(4 * 2, inf.storagespaceConsumedWithStriped(null)); - assertEquals(4 * 2, inf.storagespaceConsumed(null)); + assertEquals(4 * 2, inf.storagespaceConsumedWithStriped().getStorageSpace()); + assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace()); } @Test @@ -188,10 +194,8 @@ public void testBlockStripedComputeQuotaUsage() blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); - BlockStoragePolicySuite suite = - BlockStoragePolicySuite.createDefaultSuite(); QuotaCounts counts = - inf.computeQuotaUsageWithStriped(suite, + inf.computeQuotaUsageWithStriped(defaultPolicy, new QuotaCounts.Builder().build()); assertEquals(1, counts.getNameSpace()); // The total consumed space is the sum of @@ -215,10 +219,8 @@ public void testBlockStripedUCComputeQuotaUsage() bInfoStripedUC.setNumBytes(100); inf.addBlock(bInfoStripedUC); - BlockStoragePolicySuite suite - = BlockStoragePolicySuite.createDefaultSuite(); QuotaCounts counts - = inf.computeQuotaUsageWithStriped(suite, + = inf.computeQuotaUsageWithStriped(defaultPolicy, new QuotaCounts.Builder().build()); assertEquals(1024, inf.getPreferredBlockSize()); assertEquals(1, counts.getNameSpace()); From 54d28275226e0bc3b0d46142ce20f74a52e66395 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 12 May 2015 14:31:28 -0700 Subject: [PATCH 109/212] HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be handled properly. Contributed by Rakesh R. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../main/java/org/apache/hadoop/hdfs/DFSClient.java | 12 +++++++----- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index f026a5cde76c3..79ad208aaf294 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -201,3 +201,6 @@ HDFS-8372. Erasure coding: compute storage type quotas for striped files, to be consistent with HDFS-8327. (Zhe Zhang via jing9) + + HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be + handled properly (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 9155b4d09ec63..ffeb568cff837 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1193,12 +1193,14 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) // Get block info from namenode TraceScope scope = getPathTraceScope("newDFSInputStream", src); try { - ECSchema schema = getFileInfo(src).getECSchema(); - if (schema != null) { - return new DFSStripedInputStream(this, src, verifyChecksum, schema); - } else { - return new DFSInputStream(this, src, verifyChecksum); + HdfsFileStatus fileInfo = getFileInfo(src); + if (fileInfo != null) { + ECSchema schema = fileInfo.getECSchema(); + if (schema != null) { + return new DFSStripedInputStream(this, src, verifyChecksum, schema); + } } + return new DFSInputStream(this, src, verifyChecksum); } finally { scope.close(); } From 8da9e18dadf536d340d5f291a105e48380d0282e Mon Sep 17 00:00:00 2001 From: yliu Date: Wed, 13 May 2015 08:48:56 +0800 Subject: [PATCH 110/212] HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 ++ .../apache/hadoop/hdfs/DFSStripedInputStream.java | 15 ++++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 79ad208aaf294..0a2bb9e1d74d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -204,3 +204,5 @@ HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be handled properly (Rakesh R via zhz) + + HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 7678fae0672ac..8f15edad75127 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -130,12 +130,12 @@ boolean include(long pos) { } } - private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS; - private final BlockReader[] blockReaders = new BlockReader[groupSize]; - private final DatanodeInfo[] currentNodes = new DatanodeInfo[groupSize]; + private final BlockReader[] blockReaders; + private final DatanodeInfo[] currentNodes; private final int cellSize; private final short dataBlkNum; private final short parityBlkNum; + private final short groupSize; /** the buffer for a complete stripe */ private ByteBuffer curStripeBuf; private final ECSchema schema; @@ -155,6 +155,9 @@ boolean include(long pos) { cellSize = schema.getChunkSize(); dataBlkNum = (short) schema.getNumDataUnits(); parityBlkNum = (short) schema.getNumParityUnits(); + groupSize = dataBlkNum; + blockReaders = new BlockReader[groupSize]; + currentNodes = new DatanodeInfo[groupSize]; curStripeRange = new StripeRange(0, 0); readingService = new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); @@ -391,6 +394,12 @@ private int getStripedBufOffset(long offsetInBlockGroup) { return (int) (offsetInBlockGroup % stripeLen); } + @Override + public synchronized boolean seekToNewSource(long targetPos) + throws IOException { + return false; + } + @Override protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException { From 9798065cbb5cf5de94fe8e17ac22388f70e12dd6 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 12 May 2015 23:10:25 -0700 Subject: [PATCH 111/212] HDFS-8195. Erasure coding: Fix file quota change when we complete/commit the striped blocks. Contributed by Takuya Fukudome. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hdfs/server/namenode/FSDirectory.java | 2 +- .../hdfs/server/namenode/FSNamesystem.java | 25 +++- .../namenode/TestQuotaWithStripedBlocks.java | 125 ++++++++++++++++++ 4 files changed, 151 insertions(+), 4 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0a2bb9e1d74d7..0945d72268a3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -206,3 +206,6 @@ handled properly (Rakesh R via zhz) HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu) + + HDFS-8195. Erasure coding: Fix file quota change when we complete/commit + the striped blocks. (Takuya Fukudome via zhz) 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 b830157f7a26c..7989ccb6164c1 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 @@ -520,7 +520,7 @@ void updateCount(INodesInPath iip, long nsDelta, long ssDelta, short replication final INodeFile fileINode = iip.getLastINode().asFile(); EnumCounters typeSpaceDeltas = getStorageTypeDeltas(fileINode.getStoragePolicyID(), ssDelta, - replication, replication);; + replication, replication); updateCount(iip, iip.length() - 1, new QuotaCounts.Builder().nameSpace(nsDelta).storageSpace(ssDelta * replication). typeSpaces(typeSpaceDeltas).build(), 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 3179b938452dc..3c2c9cfa74422 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 @@ -3656,11 +3656,30 @@ void commitOrCompleteLastBlock( } // Adjust disk space consumption if required - // TODO: support EC files - final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes(); + final long diff; + final short replicationFactor; + if (fileINode.isStriped()) { + final ECSchema ecSchema = dir.getECSchema(iip); + final short numDataUnits = (short) ecSchema.getNumDataUnits(); + final short numParityUnits = (short) ecSchema.getNumParityUnits(); + + final long numBlocks = numDataUnits + numParityUnits; + final long fullBlockGroupSize = + fileINode.getPreferredBlockSize() * numBlocks; + + final BlockInfoStriped striped = new BlockInfoStriped(commitBlock, + numDataUnits, numParityUnits); + final long actualBlockGroupSize = striped.spaceConsumed(); + + diff = fullBlockGroupSize - actualBlockGroupSize; + replicationFactor = (short) 1; + } else { + diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes(); + replicationFactor = fileINode.getFileReplication(); + } if (diff > 0) { try { - dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication()); + dir.updateSpaceConsumed(iip, 0, -diff, replicationFactor); } catch (IOException e) { LOG.warn("Unexpected exception while updating disk space.", e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java new file mode 100644 index 0000000000000..86fcb88bd9a53 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +/** + * Make sure we correctly update the quota usage with the striped blocks. + */ +public class TestQuotaWithStripedBlocks { + private static final int BLOCK_SIZE = 1024 * 1024; + private static final long DISK_QUOTA = BLOCK_SIZE * 10; + private static final ECSchema ecSchema = + ErasureCodingSchemaManager.getSystemDefaultSchema(); + private static final int NUM_DATA_BLOCKS = ecSchema.getNumDataUnits(); + private static final int NUM_PARITY_BLOCKS = ecSchema.getNumParityUnits(); + private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + private static final Path ecDir = new Path("/ec"); + + private MiniDFSCluster cluster; + private FSDirectory dir; + private DistributedFileSystem dfs; + + @Before + public void setUp() throws IOException { + final Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE).build(); + cluster.waitActive(); + + dir = cluster.getNamesystem().getFSDirectory(); + dfs = cluster.getFileSystem(); + + dfs.mkdirs(ecDir); + dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema); + dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA); + dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA); + dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testUpdatingQuotaCount() throws Exception { + final Path file = new Path(ecDir, "file"); + FSDataOutputStream out = null; + + try { + out = dfs.create(file, (short) 1); + + INodeFile fileNode = dir.getINode4Write(file.toString()).asFile(); + ExtendedBlock previous = null; + // Create striped blocks which have a cell in each block. + Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(), + dfs, cluster.getNamesystem(), file.toString(), fileNode, + dfs.getClient().getClientName(), previous, 1); + previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(), + newBlock); + + final INodeDirectory dirNode = dir.getINode4Write(ecDir.toString()) + .asDirectory(); + final long spaceUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getStorageSpace(); + final long diskUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK); + // When we add a new block we update the quota using the full block size. + Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, spaceUsed); + Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, diskUsed); + + dfs.getClient().getNamenode().complete(file.toString(), + dfs.getClient().getClientName(), previous, fileNode.getId()); + + final long actualSpaceUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getStorageSpace(); + final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK); + // In this case the file's real size is cell size * block group size. + Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, + actualSpaceUsed); + Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, + actualDiskUsed); + } finally { + IOUtils.cleanup(null, out); + } + } +} From bba15e06d29d79ab1c379f258e5efb13a1ea39ae Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 13 May 2015 12:43:39 +0530 Subject: [PATCH 112/212] HDFS-8364. Erasure coding: fix some minor bugs in EC CLI (Contributed by Walter Su) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../hadoop-hdfs/src/main/bin/hdfs | 1 + .../hdfs/tools/erasurecode/ECCommand.java | 12 ++++--- .../test/resources/testErasureCodingConf.xml | 35 +++++++++++++++++++ 4 files changed, 47 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0945d72268a3c..190ddd64a187a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -209,3 +209,6 @@ HDFS-8195. Erasure coding: Fix file quota change when we complete/commit the striped blocks. (Takuya Fukudome via zhz) + + HDFS-8364. Erasure coding: fix some minor bugs in EC CLI + (Walter Su via vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs index 84c79b8a2f337..5ee7f4d59f45a 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs @@ -28,6 +28,7 @@ function hadoop_usage echo " datanode run a DFS datanode" echo " dfs run a filesystem command on the file system" echo " dfsadmin run a DFS admin client" + echo " erasurecode configure HDFS erasure coding zones" echo " fetchdt fetch a delegation token from the NameNode" echo " fsck run a DFS filesystem checking utility" echo " getconf get config values from configuration" diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index d53844da7cfca..2b6a6a52062e4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -135,7 +135,7 @@ protected void processPath(PathData item) throws IOException { out.println("EC Zone created successfully at " + item.path); } catch (IOException e) { throw new IOException("Unable to create EC zone for the path " - + item.path, e); + + item.path + ". " + e.getMessage()); } } } @@ -165,10 +165,14 @@ protected void processPath(PathData item) throws IOException { DistributedFileSystem dfs = (DistributedFileSystem) item.fs; try { ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path); - out.println(ecZoneInfo.toString()); + if (ecZoneInfo != null) { + out.println(ecZoneInfo.toString()); + } else { + out.println("Path " + item.path + " is not in EC zone"); + } } catch (IOException e) { - throw new IOException("Unable to create EC zone for the path " - + item.path, e); + throw new IOException("Unable to get EC zone for the path " + + item.path + ". " + e.getMessage()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml index b7b29d3a6961c..66892f0a019a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -122,6 +122,24 @@ + + createZone : create a zone twice + + -fs NAMENODE -mkdir /eczone + -fs NAMENODE -createZone /eczone + -fs NAMENODE -createZone /eczone + + + -fs NAMENODE -rmdir /eczone + + + + SubstringComparator + Directory /eczone is already in an erasure coding zone + + + + createZone : default schema @@ -140,6 +158,23 @@ + + getZoneInfo : get information about the EC zone at specified path not in zone + + -fs NAMENODE -mkdir /noec + -fs NAMENODE -getZoneInfo /noec + + + -fs NAMENODE -rmdir /noec + + + + SubstringComparator + Path NAMENODE/noec is not in EC zone + + + + getZoneInfo : get information about the EC zone at specified path From c99c3379282779e11ebda88d845bb89407f2f350 Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Thu, 14 May 2015 11:27:48 +0530 Subject: [PATCH 113/212] HDFS-8391. NN should consider current EC tasks handling count from DN while assigning new tasks. Contributed by Uma Maheswara Rao G. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hadoop/hdfs/server/datanode/DataNode.java | 19 +++++++++++++++++-- .../erasurecode/ErasureCodingWorker.java | 4 +++- 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 190ddd64a187a..1456434231d89 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -212,3 +212,6 @@ HDFS-8364. Erasure coding: fix some minor bugs in EC CLI (Walter Su via vinayakumarb) + + HDFS-8391. NN should consider current EC tasks handling count from DN while + assigning new tasks. (umamahesh) 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 5eca2c7e98650..a1a80ee3fd4fb 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 @@ -1909,6 +1909,21 @@ void incrDatanodeNetworkErrors(String host) { int getXmitsInProgress() { return xmitsInProgress.get(); } + + /** + * Increments the xmitsInProgress count. xmitsInProgress count represents the + * number of data replication/reconstruction tasks running currently. + */ + public void incrementXmitsInProgress() { + xmitsInProgress.getAndIncrement(); + } + + /** + * Decrements the xmitsInProgress count + */ + public void decrementXmitsInProgress() { + xmitsInProgress.getAndDecrement(); + } private void reportBadBlock(final BPOfferService bpos, final ExtendedBlock block, final String msg) { @@ -2128,7 +2143,7 @@ private class DataTransfer implements Runnable { */ @Override public void run() { - xmitsInProgress.getAndIncrement(); + incrementXmitsInProgress(); Socket sock = null; DataOutputStream out = null; DataInputStream in = null; @@ -2207,7 +2222,7 @@ public void run() { // check if there are any disk problem checkDiskErrorAsync(); } finally { - xmitsInProgress.getAndDecrement(); + decrementXmitsInProgress(); IOUtils.closeStream(blockSender); IOUtils.closeStream(out); IOUtils.closeStream(in); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index eedb1914ceb53..7b3c24d35a878 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -312,6 +312,7 @@ private long getBlockLen(ExtendedBlock blockGroup, int i) { @Override public void run() { + datanode.incrementXmitsInProgress(); try { // Store the indices of successfully read source // This will be updated after doing real read. @@ -397,8 +398,9 @@ public void run() { // Currently we don't check the acks for packets, this is similar as // block replication. } catch (Throwable e) { - LOG.warn("Failed to recover striped block: " + blockGroup); + LOG.warn("Failed to recover striped block: " + blockGroup, e); } finally { + datanode.decrementXmitsInProgress(); // close block readers for (StripedReader stripedReader : stripedReaders) { closeBlockReader(stripedReader.blockReader); From d8ea443af0b1c8289a1dd738945831ff8be0e9c1 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Sat, 16 May 2015 16:57:12 -0700 Subject: [PATCH 114/212] Merge HDFS-8394 from trunk: Move getAdditionalBlock() and related functionalities into a separate class. --- .../java/org/apache/hadoop/hdfs/DFSUtil.java | 2 +- .../blockmanagement/BlockInfoContiguous.java | 2 +- .../server/blockmanagement/BlockManager.java | 8 +- .../server/namenode/FSDirWriteFileOp.java | 111 ++++++++++-------- .../hdfs/server/namenode/FSNamesystem.java | 6 +- .../hdfs/server/namenode/INodeFile.java | 8 -- .../hadoop/hdfs/util/StripedBlockUtil.java | 2 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 2 +- 8 files changed, 78 insertions(+), 63 deletions(-) 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 2e2ecfd652a21..c06a4355d0ff8 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 @@ -1514,7 +1514,7 @@ public static int getIoFileBufferSize(Configuration conf) { public static int getSmallBufferSize(Configuration conf) { return Math.min(getIoFileBufferSize(conf) / 2, 512); } - + /** * Probe for HDFS Encryption being enabled; this uses the value of * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI}, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index eeab076d5769d..416091fba5b27 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -42,7 +42,7 @@ public BlockInfoContiguous(Block blk, short size) { * @param from BlockReplicationInfo to copy from. */ protected BlockInfoContiguous(BlockInfoContiguous from) { - this(from, from.getBlockCollection().getBlockReplication()); + this(from, from.getBlockCollection().getPreferredBlockReplication()); this.triplets = new Object[from.triplets.length]; this.setBlockCollection(from.getBlockCollection()); } 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 5b876f951750e..8b5144821eff2 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 @@ -3567,6 +3567,11 @@ public DatanodeStorageInfo[] getStorages(BlockInfo block) { return storages; } + /** @return an iterator of the datanodes. */ + public Iterable getStorages(final Block block) { + return blocksMap.getStorages(block); + } + public int getTotalBlocks() { return blocksMap.size(); } @@ -3958,7 +3963,7 @@ public static LocatedStripedBlock newLocatedStripedBlock( null); } - public LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, + public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, DatanodeStorageInfo[] locs, long offset) throws IOException { final LocatedBlock lb; if (info.isStriped()) { @@ -3968,7 +3973,6 @@ public LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, } else { lb = newLocatedBlock(eb, locs, offset, false); } - setBlockToken(lb, BlockTokenIdentifier.AccessMode.WRITE); return lb; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index fa17d9df9b024..81a2fa7355eba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -45,10 +45,13 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.Node; import org.apache.hadoop.net.NodeBase; import org.apache.hadoop.util.ChunkedArrayList; @@ -74,7 +77,7 @@ static boolean unprotectedRemoveBlock( Block block) throws IOException { // modify file-> block and blocksMap // fileNode should be under construction - BlockInfoContiguousUnderConstruction uc = fileNode.removeLastBlock(block); + BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block); if (uc == null) { return false; } @@ -88,7 +91,7 @@ static boolean unprotectedRemoveBlock( // update space consumed fsd.updateCount(iip, 0, -fileNode.getPreferredBlockSize(), - fileNode.getPreferredBlockReplication(), true); + fileNode.getPreferredBlockReplication(), true); return true; } @@ -168,7 +171,7 @@ static ValidateAddBlockResult validateAddBlock( String src, long fileId, String clientName, ExtendedBlock previous, LocatedBlock[] onRetryBlock) throws IOException { final long blockSize; - final int replication; + final short numTargets; final byte storagePolicyID; String clientMachine; @@ -196,18 +199,21 @@ static ValidateAddBlockResult validateAddBlock( blockSize = pendingFile.getPreferredBlockSize(); clientMachine = pendingFile.getFileUnderConstructionFeature() .getClientMachine(); - replication = pendingFile.getFileReplication(); + boolean isStriped = pendingFile.isStriped(); + numTargets = isStriped ? + HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : + pendingFile.getFileReplication(); storagePolicyID = pendingFile.getStoragePolicyID(); - return new ValidateAddBlockResult(blockSize, replication, storagePolicyID, - clientMachine); + return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID, + clientMachine); } - static LocatedBlock makeLocatedBlock(FSNamesystem fsn, Block blk, + static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk, DatanodeStorageInfo[] locs, long offset) throws IOException { LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk), - locs, offset, false); + blk, locs, offset); fsn.getBlockManager().setBlockToken(lBlk, - BlockTokenIdentifier.AccessMode.WRITE); + BlockTokenIdentifier.AccessMode.WRITE); return lBlk; } @@ -236,9 +242,10 @@ static LocatedBlock storeAllocatedBlock(FSNamesystem fsn, String src, return onRetryBlock[0]; } else { // add new chosen targets to already allocated block and return - BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock(); - ((BlockInfoContiguousUnderConstruction) lastBlockInFile) - .setExpectedLocations(targets); + BlockInfo lastBlockInFile = pendingFile.getLastBlock(); + final BlockInfoUnderConstruction uc + = (BlockInfoUnderConstruction)lastBlockInFile; + uc.setExpectedLocations(targets); offset = pendingFile.computeFileSize(); return makeLocatedBlock(fsn, lastBlockInFile, targets, offset); } @@ -249,15 +256,17 @@ static LocatedBlock storeAllocatedBlock(FSNamesystem fsn, String src, ExtendedBlock.getLocalBlock(previous)); // allocate new block, record block locations in INode. - Block newBlock = fsn.createNewBlock(); + final boolean isStriped = pendingFile.isStriped(); + // allocate new block, record block locations in INode. + Block newBlock = fsn.createNewBlock(isStriped); INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile); - saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets); + saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped); persistNewBlock(fsn, src, pendingFile); offset = pendingFile.computeFileSize(); // Return located block - return makeLocatedBlock(fsn, newBlock, targets, offset); + return makeLocatedBlock(fsn, fsn.getStoredBlock(newBlock), targets, offset); } static DatanodeStorageInfo[] chooseTargetForNewBlock( @@ -278,7 +287,7 @@ static DatanodeStorageInfo[] chooseTargetForNewBlock( : Arrays.asList(favoredNodes); // choose targets for the new block to be allocated. - return bm.chooseTarget4NewBlock(src, r.replication, clientNode, + return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode, excludedNodesSet, r.blockSize, favoredNodesList, r.storagePolicyID); } @@ -504,25 +513,38 @@ static INodeFile addFileForEditLog( /** * Add a block to the file. Returns a reference to the added block. */ - private static BlockInfoContiguous addBlock( - FSDirectory fsd, String path, INodesInPath inodesInPath, Block block, - DatanodeStorageInfo[] targets) throws IOException { + private static BlockInfo addBlock(FSDirectory fsd, String path, + INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets, + boolean isStriped) throws IOException { fsd.writeLock(); try { final INodeFile fileINode = inodesInPath.getLastINode().asFile(); Preconditions.checkState(fileINode.isUnderConstruction()); - // check quota limits and updated space consumed - fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), - fileINode.getPreferredBlockReplication(), true); - // associate new last block for the file - BlockInfoContiguousUnderConstruction blockInfo = - new BlockInfoContiguousUnderConstruction( - block, - fileINode.getFileReplication(), - HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, + final BlockInfo blockInfo; + if (isStriped) { + ECSchema ecSchema = fsd.getECSchema(inodesInPath); + short numDataUnits = (short) ecSchema.getNumDataUnits(); + short numParityUnits = (short) ecSchema.getNumParityUnits(); + short numLocations = (short) (numDataUnits + numParityUnits); + + // check quota limits and updated space consumed + fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), + numLocations, true); + blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits, + numParityUnits, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); + } else { + // check quota limits and updated space consumed + fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), + fileINode.getPreferredBlockReplication(), true); + + short numLocations = fileINode.getFileReplication(); + blockInfo = new BlockInfoContiguousUnderConstruction(block, + numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, + targets); + } fsd.getBlockManager().addBlockCollection(blockInfo, fileINode); fileINode.addBlock(blockInfo); @@ -576,7 +598,7 @@ private static INodesInPath addFile( private static FileState analyzeFileState( FSNamesystem fsn, String src, long fileId, String clientName, ExtendedBlock previous, LocatedBlock[] onRetryBlock) - throws IOException { + throws IOException { assert fsn.hasReadLock(); checkBlock(fsn, previous); @@ -659,8 +681,8 @@ private static FileState analyzeFileState( "allocation of a new block in " + src + ". Returning previously" + " allocated block " + lastBlockInFile); long offset = file.computeFileSize(); - BlockInfoContiguousUnderConstruction lastBlockUC = - (BlockInfoContiguousUnderConstruction) lastBlockInFile; + BlockInfoUnderConstruction lastBlockUC = + (BlockInfoUnderConstruction) lastBlockInFile; onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile, lastBlockUC.getExpectedStorageLocations(), offset); return new FileState(file, src, iip); @@ -685,14 +707,8 @@ static boolean completeFile(FSNamesystem fsn, FSPermissionChecker pc, checkBlock(fsn, last); byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); src = fsn.dir.resolvePath(pc, src, pathComponents); - boolean success = completeFileInternal(fsn, src, holder, - ExtendedBlock.getLocalBlock(last), - fileId); - if (success) { - NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg - + " is closed by " + holder); - } - return success; + return completeFileInternal(fsn, src, holder, + ExtendedBlock.getLocalBlock(last), fileId); } private static boolean completeFileInternal( @@ -794,13 +810,12 @@ private static void persistNewBlock( * @param targets target datanodes where replicas of the new block is placed * @throws QuotaExceededException If addition of block exceeds space quota */ - private static void saveAllocatedBlock( - FSNamesystem fsn, String src, INodesInPath inodesInPath, Block newBlock, - DatanodeStorageInfo[] targets) - throws IOException { + private static void saveAllocatedBlock(FSNamesystem fsn, String src, + INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets, + boolean isStriped) throws IOException { assert fsn.hasWriteLock(); - BlockInfoContiguous b = addBlock(fsn.dir, src, inodesInPath, newBlock, - targets); + BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets, + isStriped); NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src); DatanodeStorageInfo.incrementBlocksScheduled(targets); } @@ -849,15 +864,15 @@ private static class FileState { static class ValidateAddBlockResult { final long blockSize; - final int replication; + final int numTargets; final byte storagePolicyID; final String clientMachine; ValidateAddBlockResult( - long blockSize, int replication, byte storagePolicyID, + long blockSize, int numTargets, byte storagePolicyID, String clientMachine) { this.blockSize = blockSize; - this.replication = replication; + this.numTargets = numTargets; this.storagePolicyID = storagePolicyID; this.clientMachine = clientMachine; } 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 3c2c9cfa74422..71a3dcf29bb1c 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 @@ -3073,6 +3073,10 @@ boolean completeFile(final String src, String holder, writeUnlock(); } getEditLog().logSync(); + if (success) { + NameNode.stateChangeLog.info("DIR* completeFile: " + src + + " is closed by " + holder); + } return success; } @@ -3080,7 +3084,7 @@ boolean completeFile(final String src, String holder, * Create new block with a unique block id and a new generation stamp. * @param isStriped is the file under striping or contiguous layout? */ - Block createNewBlock() throws IOException { + Block createNewBlock(boolean isStriped) throws IOException { assert hasWriteLock(); Block b = new Block(nextBlockId(isStriped), 0, 0); // Increment the generation stamp for every new block. 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 154198c5206d7..41287e87cf7f0 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 @@ -905,14 +905,6 @@ public final QuotaCounts storagespaceConsumedWithReplication( return counts; } - public final short getReplication(int lastSnapshotId) { - if (lastSnapshotId != CURRENT_STATE_ID) { - return getFileReplication(lastSnapshotId); - } else { - return getBlockReplication(); - } - } - /** * Return the penultimate allocated block for this file. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index f7ae88a47372d..c95f0b4e0a1b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -87,7 +87,7 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, - bg.getStartOffset() + idxInBlockGroup, bg.isCorrupt(), + bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), null); } 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 0165189bc9cac..9f106cfcabe92 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 @@ -1915,7 +1915,7 @@ public static Block addStripedBlockToFile(List dataNodes, fileNode.getId(), null); final BlockInfo lastBlock = fileNode.getLastBlock(); - final int groupSize = fileNode.getBlockReplication(); + final int groupSize = fileNode.getPreferredBlockReplication(); assert dataNodes.size() >= groupSize; // 1. RECEIVING_BLOCK IBR for (int i = 0; i < groupSize; i++) { From c9103e9cacc67a614940e32fa87c5dbc3daa60de Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Tue, 19 May 2015 00:10:30 +0800 Subject: [PATCH 115/212] HDFS-8367 BlockInfoStriped uses EC schema. Contributed by Kai Sasaki --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/hdfs/protocolPB/PBHelper.java | 6 +-- .../blockmanagement/BlockInfoStriped.java | 24 ++++++++---- .../BlockInfoStripedUnderConstruction.java | 12 +++--- .../server/namenode/FSDirWriteFileOp.java | 4 +- .../hdfs/server/namenode/FSDirectory.java | 3 ++ .../hdfs/server/namenode/FSEditLogLoader.java | 34 ++++++++-------- .../hdfs/server/namenode/FSImageFormat.java | 10 +++-- .../server/namenode/FSImageFormatPBINode.java | 7 +++- .../server/namenode/FSImageSerialization.java | 14 ++++--- .../hdfs/server/namenode/FSNamesystem.java | 2 +- .../blockmanagement/TestBlockInfoStriped.java | 8 +++- .../server/namenode/TestFSEditLogLoader.java | 8 +++- .../hdfs/server/namenode/TestFSImage.java | 6 ++- .../server/namenode/TestStripedINodeFile.java | 39 ++++++------------- 15 files changed, 99 insertions(+), 80 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 1456434231d89..333d85f0169a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -215,3 +215,5 @@ HDFS-8391. NN should consider current EC tasks handling count from DN while assigning new tasks. (umamahesh) + + HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng) 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 94b2ff9013e67..a6a356cc9ce8d 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 @@ -203,6 +203,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand; @@ -445,9 +446,8 @@ public static Block convert(BlockProto b) { return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp()); } - public static BlockInfoStriped convert(StripedBlockProto p) { - return new BlockInfoStriped(convert(p.getBlock()), - (short) p.getDataBlockNum(), (short) p.getParityBlockNum()); + public static BlockInfoStriped convert(StripedBlockProto p, ECSchema schema) { + return new BlockInfoStriped(convert(p.getBlock()), schema); } public static StripedBlockProto convert(BlockInfoStriped blk) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index f0e52e370bf5e..d7a48a0b56420 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -19,7 +19,9 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.erasurecode.ECSchema; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @@ -39,6 +41,7 @@ public class BlockInfoStriped extends BlockInfo { private final short dataBlockNum; private final short parityBlockNum; + private final ECSchema schema; /** * Always the same size with triplets. Record the block index for each triplet * TODO: actually this is only necessary for over-replicated block. Thus can @@ -46,16 +49,17 @@ public class BlockInfoStriped extends BlockInfo { */ private byte[] indices; - public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) { - super(blk, (short) (dataBlockNum + parityBlockNum)); - indices = new byte[dataBlockNum + parityBlockNum]; + public BlockInfoStriped(Block blk, ECSchema schema) { + super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits())); + indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()]; initIndices(); - this.dataBlockNum = dataBlockNum; - this.parityBlockNum = parityBlockNum; + this.schema = schema; + this.dataBlockNum = (short)schema.getNumDataUnits(); + this.parityBlockNum = (short)schema.getNumParityUnits(); } BlockInfoStriped(BlockInfoStriped b) { - this(b, b.dataBlockNum, b.parityBlockNum); + this(b, b.getSchema()); this.setBlockCollection(b.getBlockCollection()); } @@ -71,6 +75,10 @@ public short getParityBlockNum() { return parityBlockNum; } + public ECSchema getSchema() { + return schema; + } + private void initIndices() { for (int i = 0; i < indices.length; i++) { indices[i] = -1; @@ -231,8 +239,8 @@ public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( BlockUCState s, DatanodeStorageInfo[] targets) { final BlockInfoStripedUnderConstruction ucBlock; if(isComplete()) { - ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(), - getParityBlockNum(), s, targets); + ucBlock = new BlockInfoStripedUnderConstruction(this, schema, + s, targets); ucBlock.setBlockCollection(getBlockCollection()); } else { // the block is already under construction diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index b9916156edacc..40f880f9d5200 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.IOException; @@ -56,17 +57,16 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped /** * Constructor with null storage targets. */ - public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum, - short parityBlockNum) { - this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null); + public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema) { + this(blk, schema, UNDER_CONSTRUCTION, null); } /** * Create a striped block that is currently being constructed. */ - public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum, - short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) { - super(blk, dataBlockNum, parityBlockNum); + public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, + BlockUCState state, DatanodeStorageInfo[] targets) { + super(blk, schema); assert getBlockUCState() != COMPLETE : "BlockInfoStripedUnderConstruction cannot be in COMPLETE state"; this.blockUCState = state; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 81a2fa7355eba..6137fd51a5c5a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -532,8 +532,8 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, // check quota limits and updated space consumed fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), numLocations, true); - blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits, - numParityUnits, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, + blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema, + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } else { // check quota limits and updated space consumed 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 7989ccb6164c1..bb8282d1da01a 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 @@ -54,6 +54,9 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; 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 592c421619bf3..66b2f82edf23e 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 @@ -104,6 +104,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter; import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step; import org.apache.hadoop.hdfs.util.Holder; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.ChunkedArrayList; import com.google.common.base.Joiner; @@ -416,8 +417,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip)); + updateBlocks(fsDir, addCloseOp, iip, newFile, + fsDir.getECSchema(iip), fsDir.isInECZone(iip)); break; } case OP_CLOSE: { @@ -437,8 +438,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip)); + updateBlocks(fsDir, addCloseOp, iip, file, + fsDir.getECSchema(iip), fsDir.isInECZone(iip)); // Now close the file if (!file.isUnderConstruction() && @@ -496,9 +497,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures - // TODO whether the file is striped should later be retrieved from iip - updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip)); - + updateBlocks(fsDir, updateOp, iip, oldFile, + fsDir.getECSchema(iip), fsDir.isInECZone(iip)); + if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); } @@ -514,8 +515,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // add the new block to the INodeFile - // TODO whether the file is striped should later be retrieved from iip - addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip)); + addNewBlock(addBlockOp, oldFile, + fsDir.getECSchema(iip), fsDir.isInECZone(iip)); break; } case OP_SET_REPLICATION: { @@ -956,7 +957,8 @@ private static String formatEditLogReplayError(EditLogInputStream in, /** * Add a new block into the given INodeFile */ - private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped) + private void addNewBlock(AddBlockOp op, INodeFile file, + ECSchema schema, boolean isStriped) throws IOException { BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); @@ -985,8 +987,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped) // add the new block final BlockInfo newBlockInfo; if (isStriped) { - newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, - HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); + newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, schema); } else { newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); @@ -1001,7 +1002,8 @@ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped) * @throws IOException */ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, - INodesInPath iip, INodeFile file, boolean isStriped) throws IOException { + INodesInPath iip, INodeFile file, ECSchema schema, + boolean isStriped) throws IOException { // Update its block list BlockInfo[] oldBlocks = file.getBlocks(); Block[] newBlocks = op.getBlocks(); @@ -1069,8 +1071,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // what about an old-version fsync() where fsync isn't called // until several blocks in? if (isStriped) { - newBI = new BlockInfoStripedUnderConstruction(newBlock, - HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); + newBI = new BlockInfoStripedUnderConstruction(newBlock, schema); } else { newBI = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); @@ -1080,8 +1081,9 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // is only executed when loading edits written by prior // versions of Hadoop. Current versions always log // OP_ADD operations as each block is allocated. + // TODO: ECSchema can be restored from persisted file (HDFS-7859). newBI = isStriped ? new BlockInfoStriped(newBlock, - HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) : + ErasureCodingSchemaManager.getSystemDefaultSchema()) : new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); } fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file); 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 6f485f53fecf9..3083952682036 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 @@ -76,6 +76,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.StringUtils; import com.google.common.annotations.VisibleForTesting; @@ -762,6 +763,9 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion) && (in.readBoolean()); final int numBlocks = in.readInt(); + // TODO: ECSchema can be restored from persisted file (HDFS-7859). + final ECSchema schema = isStriped ? + ErasureCodingSchemaManager.getSystemDefaultSchema() : null; if (numBlocks >= 0) { // file @@ -771,8 +775,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, if (isStriped) { blocks = new Block[numBlocks]; for (int j = 0; j < numBlocks; j++) { - blocks[j] = new BlockInfoStriped(new Block(), - HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); + blocks[j] = new BlockInfoStriped(new Block(), schema); blocks[j].readFields(in); } } else { @@ -804,8 +807,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk; blocks[blocks.length - 1] = new BlockInfoStripedUnderConstruction(lastBlk, - lastStripedBlk.getDataBlockNum(), - lastStripedBlk.getParityBlockNum()); + lastStripedBlk.getSchema()); } else { blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(lastBlk, 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 583f4b07d896c..e157b95c52198 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 @@ -69,6 +69,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.protobuf.ByteString; +import org.apache.hadoop.io.erasurecode.ECSchema; @InterfaceAudience.Private public final class FSImageFormatPBINode { @@ -354,10 +355,12 @@ private INodeFile loadINodeFile(INodeSection.INode n) { FileWithStripedBlocksFeature stripeFeature = null; if (f.hasStripedBlocks()) { + // TODO: HDFS-7859 + ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); StripedBlocksFeature sb = f.getStripedBlocks(); stripeFeature = file.addStripedBlocksFeature(); for (StripedBlockProto sp : sb.getBlocksList()) { - stripeFeature.addBlock(PBHelper.convert(sp)); + stripeFeature.addBlock(PBHelper.convert(sp, schema)); } } @@ -372,7 +375,7 @@ private INodeFile loadINodeFile(INodeSection.INode n) { if (stripeFeature != null) { BlockInfoStriped striped = (BlockInfoStriped) lastBlk; ucBlk = new BlockInfoStripedUnderConstruction(striped, - striped.getDataBlockNum(), striped.getParityBlockNum()); + striped.getSchema()); } else { ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication); 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 25febd45f3bad..47447166913ad 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 @@ -50,6 +50,7 @@ import org.apache.hadoop.io.ShortWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.xml.sax.ContentHandler; import org.xml.sax.SAXException; @@ -130,7 +131,11 @@ static INodeFile readINodeUnderConstruction( final boolean isStriped = NameNodeLayoutVersion.supports( NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion) && (in.readBoolean()); - + + // TODO: ECSchema can be restored from persisted file (HDFS-7859). + final ECSchema schema = isStriped ? + ErasureCodingSchemaManager.getSystemDefaultSchema() : null; + int numBlocks = in.readInt(); final BlockInfoContiguous[] blocksContiguous; @@ -140,15 +145,12 @@ static INodeFile readINodeUnderConstruction( blocksStriped = new BlockInfoStriped[numBlocks]; int i = 0; for (; i < numBlocks - 1; i++) { - blocksStriped[i] = new BlockInfoStriped(new Block(), - HdfsConstants.NUM_DATA_BLOCKS, - HdfsConstants.NUM_PARITY_BLOCKS); + blocksStriped[i] = new BlockInfoStriped(new Block(), schema); blocksStriped[i].readFields(in); } if (numBlocks > 0) { blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(), - HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS, - BlockUCState.UNDER_CONSTRUCTION, null); + schema, BlockUCState.UNDER_CONSTRUCTION, null); blocksStriped[i].readFields(in); } } else { 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 71a3dcf29bb1c..6643dfe813dec 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 @@ -3672,7 +3672,7 @@ void commitOrCompleteLastBlock( fileINode.getPreferredBlockSize() * numBlocks; final BlockInfoStriped striped = new BlockInfoStriped(commitBlock, - numDataUnits, numParityUnits); + ecSchema); final long actualBlockGroupSize = striped.spaceConsumed(); diff = fullBlockGroupSize - actualBlockGroupSize; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java index 3b689ebd4a5a5..2d6b5092d7a34 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -20,6 +20,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.Assert; import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; @@ -42,8 +44,10 @@ public class TestBlockInfoStriped { private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; private static final long BASE_ID = -1600; private static final Block baseBlock = new Block(BASE_ID); + private static final ECSchema testSchema + = ErasureCodingSchemaManager.getSystemDefaultSchema(); private final BlockInfoStriped info = new BlockInfoStriped(baseBlock, - NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + testSchema); private Block[] createReportedBlocks(int num) { Block[] blocks = new Block[num]; @@ -231,7 +235,7 @@ public void testWrite() { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutput out = new DataOutputStream(byteStream); BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes, - generationStamp), NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + generationStamp), testSchema); try { blk.write(out); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 57f62c9d01079..290942372d44c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.test.PathUtils; import org.apache.log4j.Level; import org.junit.Test; @@ -72,6 +73,9 @@ public class TestFSEditLogLoader { private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class); private static final int NUM_DATA_NODES = 0; + + private static final ECSchema testSchema + = ErasureCodingSchemaManager.getSystemDefaultSchema(); @Test public void testDisplayRecentEditLogOpCodes() throws IOException { @@ -459,7 +463,7 @@ public void testAddNewStripedBlock() throws IOException{ // Add a striped block to the file BlockInfoStriped stripedBlk = new BlockInfoStriped( - new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum); + new Block(blkId, blkNumBytes, timestamp), testSchema); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); file.getStripedBlocksFeature().addBlock(stripedBlk); @@ -525,7 +529,7 @@ public void testUpdateStripedBlocks() throws IOException{ Path p = new Path(testFilePath); DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); BlockInfoStriped stripedBlk = new BlockInfoStriped( - new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum); + new Block(blkId, blkNumBytes, timestamp), testSchema); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); file.getStripedBlocksFeature().addBlock(stripedBlk); 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 bb37534667253..5e015bf0c3696 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.Assert; import org.apache.hadoop.fs.permission.PermissionStatus; @@ -67,6 +68,9 @@ public class TestFSImage { private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ = "image-with-zero-block-size.tar.gz"; + private static final ECSchema testSchema + = ErasureCodingSchemaManager.getSystemDefaultSchema(); + @Test public void testPersist() throws IOException { Configuration conf = new Configuration(); @@ -158,7 +162,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con for (int i = 0; i < stripedBlks.length; i++) { stripedBlks[i] = new BlockInfoStriped( new Block(stripedBlkId + i, preferredBlockSize, timestamp), - HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS); + testSchema); file.getStripedBlocksFeature().addBlock(stripedBlks[i]); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index e24bc54a6c40b..9e4da843bd90f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -51,6 +51,9 @@ public class TestStripedINodeFile { private final BlockStoragePolicy defaultPolicy = defaultSuite.getDefaultPolicy(); + private static final ECSchema testSchema + = ErasureCodingSchemaManager.getSystemDefaultSchema(); + private static INodeFile createStripedINodeFile() { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID); @@ -69,9 +72,7 @@ public void testBlockStripedTotalBlockCount() { ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk, testSchema); assertEquals(9, blockInfoStriped.getTotalBlockNum()); } @@ -83,9 +84,7 @@ public void testBlockStripedLength() inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk, testSchema); inf.addBlock(blockInfoStriped); assertEquals(1, inf.getBlocks().length); } @@ -98,9 +97,7 @@ public void testBlockStripedConsumedSpace() inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk, testSchema); blockInfoStriped.setNumBytes(1); inf.addBlock(blockInfoStriped); // 0. Calculate the total bytes per stripes @@ -127,15 +124,11 @@ public void testMultipleBlockStripedConsumedSpace() inf.addStripedBlocksFeature(); Block blk1 = new Block(1); BlockInfoStriped blockInfoStriped1 - = new BlockInfoStriped(blk1, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk1, testSchema); blockInfoStriped1.setNumBytes(1); Block blk2 = new Block(2); BlockInfoStriped blockInfoStriped2 - = new BlockInfoStriped(blk2, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk2, testSchema); blockInfoStriped2.setNumBytes(1); inf.addBlock(blockInfoStriped1); inf.addBlock(blockInfoStriped2); @@ -152,9 +145,7 @@ public void testBlockStripedFileSize() inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk, testSchema); blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); // Compute file size should return actual data @@ -171,9 +162,7 @@ public void testBlockStripedUCFileSize() inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStripedUnderConstruction bInfoStripedUC - = new BlockInfoStripedUnderConstruction(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStripedUnderConstruction(blk, testSchema); bInfoStripedUC.setNumBytes(100); inf.addBlock(bInfoStripedUC); assertEquals(100, inf.computeFileSize()); @@ -188,9 +177,7 @@ public void testBlockStripedComputeQuotaUsage() inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStriped(blk, testSchema); blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); @@ -213,9 +200,7 @@ public void testBlockStripedUCComputeQuotaUsage() inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStripedUnderConstruction bInfoStripedUC - = new BlockInfoStripedUnderConstruction(blk, - (short)defaultSchema.getNumDataUnits(), - (short)defaultSchema.getNumParityUnits()); + = new BlockInfoStripedUnderConstruction(blk, testSchema); bInfoStripedUC.setNumBytes(100); inf.addBlock(bInfoStripedUC); From a9197269143f9d976a1565bbf4e383fac6e1326f Mon Sep 17 00:00:00 2001 From: waltersu4549 Date: Mon, 18 May 2015 19:10:37 +0800 Subject: [PATCH 116/212] HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549) --- .../hadoop/hdfs/TestWriteReadStripedFile.java | 267 ++++++++++-------- 1 file changed, 148 insertions(+), 119 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 57d6eb9a0dd8b..f78fb7a5c0cc8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -21,9 +21,13 @@ import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.web.ByteRangeInputStream; +import org.apache.hadoop.hdfs.web.WebHdfsConstants; +import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.junit.AfterClass; import org.junit.Assert; @@ -33,23 +37,26 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Random; public class TestWriteReadStripedFile { private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - - private static DistributedFileSystem fs; private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final static int stripesPerBlock = 4; static int blockSize = cellSize * stripesPerBlock; static int numDNs = dataBlocks + parityBlocks + 2; private static MiniDFSCluster cluster; + private static Configuration conf; + private static FileSystem fs; + + private static Random r= new Random(); @BeforeClass public static void setup() throws IOException { - Configuration conf = new Configuration(); + conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); @@ -134,7 +141,7 @@ public void testFileMoreThanABlockGroup1() throws IOException { @Test public void testFileMoreThanABlockGroup2() throws IOException { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", - blockSize * dataBlocks + cellSize+ 123); + blockSize * dataBlocks + cellSize + 123); } @@ -171,7 +178,7 @@ private byte getByte(long pos) { } private void assertSeekAndRead(FSDataInputStream fsdis, int pos, - int writeBytes) throws IOException { + int writeBytes) throws IOException { fsdis.seek(pos); byte[] buf = new byte[writeBytes]; int readLen = readAll(fsdis, buf); @@ -182,147 +189,169 @@ private void assertSeekAndRead(FSDataInputStream fsdis, int pos, } } - private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes) + private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) throws IOException { - Path testPath = new Path(src); - final byte[] bytes = generateBytes(writeBytes); - DFSTestUtil.writeFile(fs, testPath, new String(bytes)); - //check file length - FileStatus status = fs.getFileStatus(testPath); - long fileLength = status.getLen(); + final byte[] expected = generateBytes(fileLength); + Path srcPath = new Path(src); + DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + + verifyLength(fs, srcPath, fileLength); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); + verifySeek(fs, srcPath, fileLength); + verifyStatefulRead(fs, srcPath, fileLength, expected, + ByteBuffer.allocate(fileLength + 100)); + verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); + verifyStatefulRead(fs, srcPath, fileLength, expected, + ByteBuffer.allocate(1024)); + } + + @Test + public void testWriteReadUsingWebHdfs() throws Exception { + int fileLength = blockSize * dataBlocks + cellSize + 123; + + final byte[] expected = generateBytes(fileLength); + FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf, + WebHdfsConstants.WEBHDFS_SCHEME); + Path srcPath = new Path("/testWriteReadUsingWebHdfs_stripe"); + DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + + verifyLength(fs, srcPath, fileLength); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); + verifySeek(fs, srcPath, fileLength); + verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); + //webhdfs doesn't support bytebuffer read + + } + + void verifyLength(FileSystem fs, Path srcPath, int fileLength) + throws IOException { + FileStatus status = fs.getFileStatus(srcPath); Assert.assertEquals("File length should be the same", - writeBytes, fileLength); + fileLength, status.getLen()); + } - // pread - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - byte[] buf = new byte[writeBytes + 100]; - int readLen = fsdis.read(0, buf, 0, buf.length); - readLen = readLen >= 0 ? readLen : 0; - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), - buf[i]); + void verifyPread(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, byte[] buf) throws IOException { + FSDataInputStream in = fs.open(srcPath); + int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102, + cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102, + cellSize * dataBlocks, fileLength - 102, fileLength - 1}; + for (int startOffset : startOffsets) { + startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); + int remaining = fileLength - startOffset; + in.readFully(startOffset, buf, 0, remaining); + for (int i = 0; i < remaining; i++) { + Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + + "same", + expected[startOffset + i], buf[i]); } } + in.close(); + } - // stateful read with byte array - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - byte[] buf = new byte[writeBytes + 100]; - int readLen = readAll(fsdis, buf); - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), - buf[i]); + void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, byte[] buf) throws IOException { + FSDataInputStream in = fs.open(srcPath); + final byte[] result = new byte[fileLength]; + int readLen = 0; + int ret; + do { + ret = in.read(buf, 0, buf.length); + if (ret > 0) { + System.arraycopy(buf, 0, result, readLen, ret); + readLen += ret; } - } + } while (ret >= 0); + Assert.assertEquals("The length of file should be the same to write size", + fileLength, readLen); + Assert.assertArrayEquals(expected, result); + in.close(); + } - // seek and stateful read - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - // seek to 1/2 of content - int pos = writeBytes/2; - assertSeekAndRead(fsdis, pos, writeBytes); - // seek to 1/3 of content - pos = writeBytes/3; - assertSeekAndRead(fsdis, pos, writeBytes); + void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, ByteBuffer buf) throws IOException { + FSDataInputStream in = fs.open(srcPath); + ByteBuffer result = ByteBuffer.allocate(fileLength); + int readLen = 0; + int ret; + do { + ret = in.read(buf); + if (ret > 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } while (ret >= 0); + readLen = readLen >= 0 ? readLen : 0; + Assert.assertEquals("The length of file should be the same to write size", + fileLength, readLen); + Assert.assertArrayEquals(expected, result.array()); + in.close(); + } - // seek to 0 pos - pos = 0; - assertSeekAndRead(fsdis, pos, writeBytes); - if (writeBytes > cellSize) { - // seek to cellSize boundary - pos = cellSize -1; - assertSeekAndRead(fsdis, pos, writeBytes); - } + void verifySeek(FileSystem fs, Path srcPath, int fileLength) + throws IOException { + FSDataInputStream in = fs.open(srcPath); + // seek to 1/2 of content + int pos = fileLength / 2; + assertSeekAndRead(in, pos, fileLength); + + // seek to 1/3 of content + pos = fileLength / 3; + assertSeekAndRead(in, pos, fileLength); + + // seek to 0 pos + pos = 0; + assertSeekAndRead(in, pos, fileLength); + + if (fileLength > cellSize) { + // seek to cellSize boundary + pos = cellSize - 1; + assertSeekAndRead(in, pos, fileLength); + } - if (writeBytes > cellSize * dataBlocks) { - // seek to striped cell group boundary - pos = cellSize * dataBlocks - 1; - assertSeekAndRead(fsdis, pos, writeBytes); - } + if (fileLength > cellSize * dataBlocks) { + // seek to striped cell group boundary + pos = cellSize * dataBlocks - 1; + assertSeekAndRead(in, pos, fileLength); + } - if (writeBytes > blockSize * dataBlocks) { - // seek to striped block group boundary - pos = blockSize * dataBlocks - 1; - assertSeekAndRead(fsdis, pos, writeBytes); - } + if (fileLength > blockSize * dataBlocks) { + // seek to striped block group boundary + pos = blockSize * dataBlocks - 1; + assertSeekAndRead(in, pos, fileLength); + } + if(!(in.getWrappedStream() instanceof ByteRangeInputStream)){ try { - fsdis.seek(-1); + in.seek(-1); Assert.fail("Should be failed if seek to negative offset"); } catch (EOFException e) { // expected } try { - fsdis.seek(writeBytes + 1); + in.seek(fileLength + 1); Assert.fail("Should be failed if seek after EOF"); } catch (EOFException e) { // expected } } - - // stateful read with ByteBuffer - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100); - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf); - if (ret > 0) { - readLen += ret; - } - } while (ret >= 0); - readLen = readLen >= 0 ? readLen : 0; - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - for (int i = 0; i < writeBytes; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", getByte(i), - buf.array()[i]); - } - } - - // stateful read with 1KB size byte array - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - final byte[] result = new byte[writeBytes]; - final byte[] buf = new byte[1024]; - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf, 0, buf.length); - if (ret > 0) { - System.arraycopy(buf, 0, result, readLen, ret); - readLen += ret; - } - } while (ret >= 0); - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - Assert.assertArrayEquals(bytes, result); - } - - // stateful read using ByteBuffer with 1KB size - try (FSDataInputStream fsdis = fs.open(new Path(src))) { - final ByteBuffer result = ByteBuffer.allocate(writeBytes); - final ByteBuffer buf = ByteBuffer.allocate(1024); - int readLen = 0; - int ret; - do { - ret = fsdis.read(buf); - if (ret > 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); - } - } while (ret >= 0); - Assert.assertEquals("The length of file should be the same to write size", - writeBytes, readLen); - Assert.assertArrayEquals(bytes, result.array()); - } + in.close(); } @Test From 9c7a78c8748a55dc94f7913b5d5b2c2bc27720d5 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 18 May 2015 10:06:56 -0700 Subject: [PATCH 117/212] HADOOP-11921. Enhance tests for erasure coders. Contributed by Kai Zheng. --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/io/erasurecode/TestCoderBase.java | 50 +++++------ .../coder/TestErasureCoderBase.java | 89 ++++++++++--------- .../erasurecode/coder/TestRSErasureCoder.java | 64 +++++++++---- .../io/erasurecode/coder/TestXORCoder.java | 24 +++-- .../erasurecode/rawcoder/TestRSRawCoder.java | 76 ++++++++-------- .../rawcoder/TestRSRawCoderBase.java | 51 +++++++++++ .../rawcoder/TestRawCoderBase.java | 45 +++++----- .../erasurecode/rawcoder/TestXORRawCoder.java | 24 +++-- 9 files changed, 274 insertions(+), 151 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 97492702216ff..c10ffbdbf72e9 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -44,3 +44,5 @@ HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng) HADOOP-11841. Remove unused ecschema-def.xml files. (szetszwo) + + HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 22fd98d2c4d2e..be1924c94f655 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -49,15 +49,15 @@ public abstract class TestCoderBase { * Prepare before running the case. * @param numDataUnits * @param numParityUnits - * @param erasedIndexes + * @param erasedDataIndexes */ protected void prepare(Configuration conf, int numDataUnits, - int numParityUnits, int[] erasedIndexes) { + int numParityUnits, int[] erasedDataIndexes) { this.conf = conf; this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; - this.erasedDataIndexes = erasedIndexes != null ? - erasedIndexes : new int[] {0}; + this.erasedDataIndexes = erasedDataIndexes != null ? + erasedDataIndexes : new int[] {0}; } /** @@ -82,15 +82,19 @@ protected void compareAndVerify(ECChunk[] erasedChunks, } /** - * Adjust and return erased indexes based on the array of the input chunks ( - * parity chunks + data chunks). - * @return + * Adjust and return erased indexes altogether, including erased data indexes + * and parity indexes. + * @return erased indexes altogether */ protected int[] getErasedIndexesForDecoding() { int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length]; + + int idx = 0; + for (int i = 0; i < erasedDataIndexes.length; i++) { - erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits; + erasedIndexesForDecoding[idx ++] = erasedDataIndexes[i] + numParityUnits; } + return erasedIndexesForDecoding; } @@ -116,30 +120,23 @@ protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks, } /** - * Have a copy of the data chunks that's to be erased thereafter. The copy - * will be used to compare and verify with the to be recovered chunks. + * Erase chunks to test the recovering of them. Before erasure clone them + * first so could return them. * @param dataChunks - * @return + * @return clone of erased chunks */ - protected ECChunk[] copyDataChunksToErase(ECChunk[] dataChunks) { - ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length]; - - int j = 0; - for (int i = 0; i < erasedDataIndexes.length; i++) { - copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]); - } + protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks) { + ECChunk[] toEraseChunks = new ECChunk[erasedDataIndexes.length]; - return copiedChunks; - } + int idx = 0; - /** - * Erase some data chunks to test the recovering of them - * @param dataChunks - */ - protected void eraseSomeDataBlocks(ECChunk[] dataChunks) { for (int i = 0; i < erasedDataIndexes.length; i++) { - eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]); + ECChunk chunk = dataChunks[erasedDataIndexes[i]]; + toEraseChunks[idx ++] = cloneChunkWithData(chunk); + eraseDataFromChunk(chunk); } + + return toEraseChunks; } /** @@ -277,6 +274,7 @@ protected ECChunk[] prepareParityChunksForEncoding() { */ protected ECChunk[] prepareOutputChunksForDecoding() { ECChunk[] chunks = new ECChunk[erasedDataIndexes.length]; + for (int i = 0; i < chunks.length; i++) { chunks[i] = allocateOutputChunk(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index 05a62a79f49cb..fdd0b50e282a9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -29,6 +29,9 @@ public abstract class TestErasureCoderBase extends TestCoderBase { protected Class encoderClass; protected Class decoderClass; + private ErasureCoder encoder; + private ErasureCoder decoder; + protected int numChunksInBlock = 16; /** @@ -54,39 +57,27 @@ public TestBlock(ECChunk[] chunks) { */ protected void testCoding(boolean usingDirectBuffer) { this.usingDirectBuffer = usingDirectBuffer; - - ErasureCoder encoder = createEncoder(); + prepareCoders(); // Generate data and encode ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); // Backup all the source chunks for later recovering because some coders // may affect the source data. - TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[]) - blockGroup.getDataBlocks()); - // Make a copy of a strip for later comparing - TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks); + TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[]) blockGroup.getDataBlocks()); ErasureCodingStep codingStep; - try { - codingStep = encoder.calculateCoding(blockGroup); - performCodingStep(codingStep); - } finally { - encoder.release(); - } - // Erase the copied sources - eraseSomeDataBlocks(clonedDataBlocks); + codingStep = encoder.calculateCoding(blockGroup); + performCodingStep(codingStep); + // Erase specified sources but return copies of them for later comparing + TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks); - //Decode + // Decode blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks()); - ErasureCoder decoder = createDecoder(); - try { - codingStep = decoder.calculateCoding(blockGroup); - performCodingStep(codingStep); - } finally { - decoder.release(); - } - //Compare - compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks()); + codingStep = decoder.calculateCoding(blockGroup); + performCodingStep(codingStep); + + // Compare + compareAndVerify(backupBlocks, codingStep.getOutputBlocks()); } /** @@ -129,8 +120,7 @@ private void performCodingStep(ErasureCodingStep codingStep) { protected void compareAndVerify(ECBlock[] erasedBlocks, ECBlock[] recoveredBlocks) { for (int i = 0; i < erasedBlocks.length; ++i) { - compareAndVerify(((TestBlock) erasedBlocks[i]).chunks, - ((TestBlock) recoveredBlocks[i]).chunks); + compareAndVerify(((TestBlock) erasedBlocks[i]).chunks, ((TestBlock) recoveredBlocks[i]).chunks); } } @@ -151,6 +141,16 @@ private ErasureCoder createEncoder() { return encoder; } + private void prepareCoders() { + if (encoder == null) { + encoder = createEncoder(); + } + + if (decoder == null) { + decoder = createDecoder(); + } + } + /** * Create the erasure decoder for the test. * @return @@ -201,6 +201,26 @@ protected ECBlock generateDataBlock() { return new TestBlock(chunks); } + /** + * Erase blocks to test the recovering of them. Before erasure clone them + * first so could return themselves. + * @param dataBlocks + * @return clone of erased dataBlocks + */ + protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks) { + TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length]; + + int idx = 0; + + for (int i = 0; i < erasedDataIndexes.length; i++) { + TestBlock block = dataBlocks[erasedDataIndexes[i]]; + toEraseBlocks[idx ++] = cloneBlockWithData(block); + eraseDataFromBlock(block); + } + + return toEraseBlocks; + } + /** * Copy those data blocks that's to be erased for later comparing and * verifying. @@ -255,22 +275,9 @@ protected static TestBlock cloneBlockWithData(TestBlock block) { } /** - * Erase some data blocks specified by the indexes from the data blocks. - * @param dataBlocks - */ - protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) { - for (int i = 0; i < erasedDataIndexes.length; ++i) { - eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]); - } - } - - /** - * Erase data from a block specified by erased index. - * @param blocks - * @param erasedIndex + * Erase data from a block. */ - protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) { - TestBlock theBlock = blocks[erasedIndex]; + protected void eraseDataFromBlock(TestBlock theBlock) { eraseDataFromChunks(theBlock.chunks); theBlock.setErased(true); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java index 3507dd2cb7b8f..7d9d37a0a2546 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java @@ -40,19 +40,18 @@ public void setup() { } @Test - public void testCodingNoDirectBuffer_10x4() { - prepare(null, 10, 4, null); + public void testCodingNoDirectBuffer_10x4_erasing_d0() { + prepare(null, 10, 4, new int[] {0}); + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + testCoding(false); testCoding(false); } @Test - public void testCodingDirectBuffer_10x4() { - prepare(null, 10, 4, null); - testCoding(true); - } - - @Test - public void testCodingDirectBufferWithConf_10x4() { + public void testCodingDirectBufferWithConf_10x4_erasing_d0() { /** * This tests if the two configuration items work or not. */ @@ -61,31 +60,62 @@ public void testCodingDirectBufferWithConf_10x4() { RSRawErasureCoderFactory.class.getCanonicalName()); conf.setBoolean( CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false); - prepare(conf, 10, 4, null); + + prepare(conf, 10, 4, new int[]{0}); + testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasure_of_2_4() { + public void testCodingDirectBuffer_10x4_erasing_d2() { + prepare(null, 10, 4, new int[] {2}); + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_d0() { + prepare(null, 10, 4, new int[] {0}); + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingBothBuffers_10x4_erasing_d0() { + prepare(null, 10, 4, new int[] {0}); + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ + testCoding(true); + testCoding(false); + testCoding(true); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() { prepare(null, 10, 4, new int[] {2, 4}); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasing_all() { - prepare(null, 10, 4, new int[] {0, 1, 2, 3}); + public void testCodingDirectBuffer_10x4_erasing_d0_d1() { + prepare(null, 10, 4, new int[] {0, 1}); testCoding(true); } @Test - public void testCodingNoDirectBuffer_3x3() { - prepare(null, 3, 3, null); + public void testCodingNoDirectBuffer_3x3_erasing_d0() { + prepare(null, 3, 3, new int[] {0}); testCoding(false); } @Test - public void testCodingDirectBuffer_3x3() { - prepare(null, 3, 3, null); + public void testCodingDirectBuffer_3x3_erasing_d0() { + prepare(null, 3, 3, new int[] {0}); testCoding(true); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java index 109e46ebb3804..87aa656feac67 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java @@ -32,19 +32,33 @@ public void setup() { this.numDataUnits = 10; this.numParityUnits = 1; - this.erasedDataIndexes = new int[] {0}; - this.numChunksInBlock = 10; } @Test - public void testCodingNoDirectBuffer() { + public void testCodingNoDirectBuffer_erasing_d0() { + prepare(null, 10, 1, new int[] {0}); + + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + testCoding(false); testCoding(false); } @Test - public void testCodingDirectBuffer() { + public void testCodingBothBuffers_erasing_d5() { + prepare(null, 10, 1, new int[]{5}); + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ testCoding(true); + testCoding(false); + testCoding(true); + testCoding(false); } - } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 8bb5d0fc3183f..9ba3e88ff1e30 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -17,26 +17,13 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; -import org.apache.hadoop.io.erasurecode.ECChunk; -import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; import org.junit.Before; import org.junit.Test; -import java.nio.ByteBuffer; - /** - * Test raw Reed-solomon encoding and decoding. + * Test raw Reed-solomon coder implemented in Java. */ -public class TestRSRawCoder extends TestRawCoderBase { - - private static int symbolSize = 0; - private static int symbolMax = 0; - - static { - symbolSize = (int) Math.round(Math.log( - RSUtil.GF.getFieldSize()) / Math.log(2)); - symbolMax = (int) Math.pow(2, symbolSize); - } +public class TestRSRawCoder extends TestRSRawCoderBase { @Before public void setup() { @@ -45,49 +32,66 @@ public void setup() { } @Test - public void testCodingNoDirectBuffer_10x4() { - prepare(null, 10, 4, null); + public void testCodingNoDirectBuffer_10x4_erasing_d0() { + prepare(null, 10, 4, new int[] {0}); + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + testCoding(false); testCoding(false); } @Test - public void testCodingDirectBuffer_10x4() { - prepare(null, 10, 4, null); + public void testCodingDirectBuffer_10x4_erasing_d2() { + prepare(null, 10, 4, new int[] {2}); + testCoding(true); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasure_of_2_4() { - prepare(null, 10, 4, new int[] {2, 4}); + public void testCodingDirectBuffer_10x4_erasing_d0() { + prepare(null, 10, 4, new int[] {0}); + testCoding(true); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasing_all() { - prepare(null, 10, 4, new int[] {0, 1, 2, 3}); + public void testCodingBothBuffers_10x4_erasing_d0() { + prepare(null, 10, 4, new int[] {0}); + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ + testCoding(true); + testCoding(false); testCoding(true); + testCoding(false); } @Test - public void testCodingNoDirectBuffer_3x3() { - prepare(null, 3, 3, null); - testCoding(false); + public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() { + prepare(null, 10, 4, new int[] {2, 4}); + testCoding(true); } @Test - public void testCodingDirectBuffer_3x3() { - prepare(null, 3, 3, null); + public void testCodingDirectBuffer_10x4_erasing_d0_d1() { + prepare(null, 10, 4, new int[] {0, 1}); testCoding(true); } - @Override - protected ECChunk generateDataChunk() { - ByteBuffer buffer = allocateOutputBuffer(); - for (int i = 0; i < chunkSize; i++) { - buffer.put((byte) RAND.nextInt(symbolMax)); - } - buffer.flip(); + @Test + public void testCodingNoDirectBuffer_3x3_erasing_d0() { + prepare(null, 3, 3, new int[] {0}); + testCoding(false); + } - return new ECChunk(buffer); + @Test + public void testCodingDirectBuffer_3x3_erasing_d0() { + prepare(null, 3, 3, new int[] {0}); + testCoding(true); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java new file mode 100644 index 0000000000000..f9e8a6baba11d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java @@ -0,0 +1,51 @@ +/** + * 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.io.erasurecode.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; +import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + * Test base for raw Reed-solomon coders. + */ +public abstract class TestRSRawCoderBase extends TestRawCoderBase { + + private static int symbolSize = 0; + private static int symbolMax = 0; + + static { + symbolSize = (int) Math.round(Math.log( + RSUtil.GF.getFieldSize()) / Math.log(2)); + symbolMax = (int) Math.pow(2, symbolSize); + } + + @Override + protected ECChunk generateDataChunk() { + ByteBuffer buffer = allocateOutputBuffer(); + for (int i = 0; i < chunkSize; i++) { + buffer.put((byte) RAND.nextInt(symbolMax)); + } + buffer.flip(); + + return new ECChunk(buffer); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index b036eed4ced15..7ba320a407987 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -26,6 +26,8 @@ public abstract class TestRawCoderBase extends TestCoderBase { protected Class encoderClass; protected Class decoderClass; + private RawErasureEncoder encoder; + private RawErasureDecoder decoder; /** * Generating source data, encoding, recovering and then verifying. @@ -37,40 +39,41 @@ public abstract class TestRawCoderBase extends TestCoderBase { */ protected void testCoding(boolean usingDirectBuffer) { this.usingDirectBuffer = usingDirectBuffer; + prepareCoders(); // Generate data and encode ECChunk[] dataChunks = prepareDataChunksForEncoding(); ECChunk[] parityChunks = prepareParityChunksForEncoding(); - RawErasureEncoder encoder = createEncoder(); // Backup all the source chunks for later recovering because some coders // may affect the source data. ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks); - // Make a copy of a strip for later comparing - ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks); - try { - encoder.encode(dataChunks, parityChunks); - } finally { - encoder.release(); - } - // Erase the copied sources - eraseSomeDataBlocks(clonedDataChunks); + encoder.encode(dataChunks, parityChunks); + + // Backup and erase some chunks + ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks); + + // Decode + ECChunk[] inputChunks = prepareInputChunksForDecoding( + clonedDataChunks, parityChunks); - //Decode - ECChunk[] inputChunks = prepareInputChunksForDecoding(clonedDataChunks, - parityChunks); ECChunk[] recoveredChunks = prepareOutputChunksForDecoding(); - RawErasureDecoder decoder = createDecoder(); - try { - decoder.decode(inputChunks, - getErasedIndexesForDecoding(), recoveredChunks); - } finally { - decoder.release(); + + decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks); + + // Compare + compareAndVerify(backupChunks, recoveredChunks); + } + + private void prepareCoders() { + if (encoder == null) { + encoder = createEncoder(); } - //Compare - compareAndVerify(toEraseDataChunks, recoveredChunks); + if (decoder == null) { + decoder = createDecoder(); + } } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java index e66e48d76a60b..62ce4fbf36fef 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java @@ -32,18 +32,32 @@ public void setup() { this.numDataUnits = 10; this.numParityUnits = 1; - - this.erasedDataIndexes = new int[] {0}; } @Test - public void testCodingNoDirectBuffer() { + public void testCodingNoDirectBuffer_erasing_d0() { + prepare(null, 10, 1, new int[] {0}); + + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + testCoding(false); testCoding(false); } @Test - public void testCodingDirectBuffer() { + public void testCodingBothBuffers_erasing_d5() { + prepare(null, 10, 1, new int[]{5}); + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ testCoding(true); + testCoding(false); + testCoding(true); + testCoding(false); } - } From 09c3a375bafa481e88d1317388a73c46950164c9 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 18 May 2015 10:09:57 -0700 Subject: [PATCH 118/212] HADOOP-11920. Refactor some codes for erasure coders. Contributed by Kai Zheng. --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/fs/CommonConfigurationKeys.java | 4 -- .../apache/hadoop/io/erasurecode/ECChunk.java | 2 +- .../coder/AbstractErasureCoder.java | 6 +- .../erasurecode/coder/RSErasureDecoder.java | 40 +----------- .../rawcoder/AbstractRawErasureCoder.java | 63 ++++++++++++++++++- .../rawcoder/AbstractRawErasureDecoder.java | 54 +++++++++------- .../rawcoder/AbstractRawErasureEncoder.java | 52 +++++++-------- .../erasurecode/rawcoder/RawErasureCoder.java | 8 +-- .../rawcoder/RawErasureDecoder.java | 24 ++++--- .../erasurecode/rawcoder/XORRawDecoder.java | 24 ++----- .../erasurecode/rawcoder/XORRawEncoder.java | 6 +- .../hadoop/io/erasurecode/TestCoderBase.java | 4 +- .../erasurecode/coder/TestRSErasureCoder.java | 6 +- 14 files changed, 156 insertions(+), 139 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index c10ffbdbf72e9..a152e317fd613 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -46,3 +46,5 @@ HADOOP-11841. Remove unused ecschema-def.xml files. (szetszwo) HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang) + + HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index bd2a24b022044..3f2871b023bd9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -143,10 +143,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** Supported erasure codec classes */ public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs"; - /** Use XOR raw coder when possible for the RS codec */ - public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY = - "io.erasurecode.codec.rs.usexor"; - /** Raw coder factory for the RS codec */ public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY = "io.erasurecode.codec.rs.rawcoder"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java index 01e8f3580e505..436e13ef63f2d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -71,7 +71,7 @@ public static ByteBuffer[] toBuffers(ECChunk[] chunks) { * @param chunks * @return an array of byte array */ - public static byte[][] toArray(ECChunk[] chunks) { + public static byte[][] toArrays(ECChunk[] chunks) { byte[][] bytesArr = new byte[chunks.length][]; ByteBuffer buffer; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index 7403e35c946ff..d491570f61f86 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -90,11 +90,7 @@ public static RawErasureCoder createRawCoder(Configuration conf, throw new RuntimeException("Failed to create raw coder", e); } - if (fact != null) { - return isEncoder ? fact.createEncoder() : fact.createDecoder(); - } - - return null; + return isEncoder ? fact.createEncoder() : fact.createDecoder(); } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java index fc664a502aefd..ec7cbb5de5276 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -17,13 +17,11 @@ */ package org.apache.hadoop.io.erasurecode.coder; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder; /** * Reed-Solomon erasure decoder that decodes a block group. @@ -32,38 +30,14 @@ */ public class RSErasureDecoder extends AbstractErasureDecoder { private RawErasureDecoder rsRawDecoder; - private RawErasureDecoder xorRawDecoder; - private boolean useXorWhenPossible = true; @Override - public void setConf(Configuration conf) { - super.setConf(conf); - - if (conf != null) { - this.useXorWhenPossible = conf.getBoolean( - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true); - } - } - - @Override protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { - RawErasureDecoder rawDecoder; - ECBlock[] inputBlocks = getInputBlocks(blockGroup); ECBlock[] outputBlocks = getOutputBlocks(blockGroup); - /** - * Optimization: according to some benchmark, when only one block is erased - * and to be recovering, the most simple XOR scheme can be much efficient. - * We will have benchmark tests to verify this opt is effect or not. - */ - if (outputBlocks.length == 1 && useXorWhenPossible) { - rawDecoder = checkCreateXorRawDecoder(); - } else { - rawDecoder = checkCreateRSRawDecoder(); - } - + RawErasureDecoder rawDecoder = checkCreateRSRawDecoder(); return new ErasureDecodingStep(inputBlocks, getErasedIndexes(inputBlocks), outputBlocks, rawDecoder); } @@ -81,19 +55,9 @@ private RawErasureDecoder checkCreateRSRawDecoder() { return rsRawDecoder; } - private RawErasureDecoder checkCreateXorRawDecoder() { - if (xorRawDecoder == null) { - xorRawDecoder = new XORRawDecoder(); - xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize()); - } - return xorRawDecoder; - } - @Override public void release() { - if (xorRawDecoder != null) { - xorRawDecoder.release(); - } else if (rsRawDecoder != null) { + if (rsRawDecoder != null) { rsRawDecoder.release(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index e6f3d929daadf..2400313b42eb8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -19,6 +19,9 @@ import org.apache.hadoop.conf.Configured; +import java.nio.ByteBuffer; +import java.util.Arrays; + /** * A common class of basic facilities to be shared by encoder and decoder * @@ -27,6 +30,9 @@ public abstract class AbstractRawErasureCoder extends Configured implements RawErasureCoder { + // Hope to reset coding buffers a little faster using it + private byte[] zeroChunkBytes; + private int numDataUnits; private int numParityUnits; private int chunkSize; @@ -37,6 +43,8 @@ public void initialize(int numDataUnits, int numParityUnits, this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; this.chunkSize = chunkSize; + + zeroChunkBytes = new byte[chunkSize]; // With ZERO by default } @Override @@ -55,7 +63,7 @@ public int getChunkSize() { } @Override - public boolean preferNativeBuffer() { + public boolean preferDirectBuffer() { return false; } @@ -63,4 +71,57 @@ public boolean preferNativeBuffer() { public void release() { // Nothing to do by default } + + /** + * Convert an array of heap ByteBuffers to an array of byte array. + * @param buffers + * @return an array of byte array + */ + protected static byte[][] toArrays(ByteBuffer[] buffers) { + byte[][] bytesArr = new byte[buffers.length][]; + + ByteBuffer buffer; + for (int i = 0; i < buffers.length; i++) { + buffer = buffers[i]; + if (buffer == null) { + bytesArr[i] = null; + continue; + } + + if (buffer.hasArray()) { + bytesArr[i] = buffer.array(); + } else { + throw new IllegalArgumentException("Invalid ByteBuffer passed, " + + "expecting heap buffer"); + } + } + + return bytesArr; + } + + /** + * Ensure the buffer (either input or output) ready to read or write with ZERO + * bytes fully in chunkSize. + * @param buffer + * @return the buffer itself + */ + protected ByteBuffer resetBuffer(ByteBuffer buffer) { + buffer.clear(); + buffer.put(zeroChunkBytes); + buffer.position(0); + + return buffer; + } + + /** + * Ensure the buffer (either input or output) ready to read or write with ZERO + * bytes fully in chunkSize. + * @param buffer bytes array buffer + * @return the buffer itself + */ + protected byte[] resetBuffer(byte[] buffer) { + System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length); + + return buffer; + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java index 4613b25eb13f3..b247543fd326a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -32,27 +32,30 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder @Override public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { - if (erasedIndexes.length == 0) { - return; - } + checkParameters(inputs, erasedIndexes, outputs); - doDecode(inputs, erasedIndexes, outputs); + boolean hasArray = inputs[0].hasArray(); + if (hasArray) { + byte[][] newInputs = toArrays(inputs); + byte[][] newOutputs = toArrays(outputs); + doDecode(newInputs, erasedIndexes, newOutputs); + } else { + doDecode(inputs, erasedIndexes, outputs); + } } /** - * Perform the real decoding using ByteBuffer - * @param inputs + * Perform the real decoding using Direct ByteBuffer. + * @param inputs Direct ByteBuffers expected * @param erasedIndexes - * @param outputs + * @param outputs Direct ByteBuffers expected */ protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs); @Override public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) { - if (erasedIndexes.length == 0) { - return; - } + checkParameters(inputs, erasedIndexes, outputs); doDecode(inputs, erasedIndexes, outputs); } @@ -69,25 +72,32 @@ protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes, @Override public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs) { - doDecode(inputs, erasedIndexes, outputs); + ByteBuffer[] newInputs = ECChunk.toBuffers(inputs); + ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs); + decode(newInputs, erasedIndexes, newOutputs); } /** - * Perform the real decoding using chunks + * Check and validate decoding parameters, throw exception accordingly. The + * checking assumes it's a MDS code. Other code can override this. * @param inputs * @param erasedIndexes * @param outputs */ - protected void doDecode(ECChunk[] inputs, int[] erasedIndexes, - ECChunk[] outputs) { - if (inputs[0].getBuffer().hasArray()) { - byte[][] inputBytesArr = ECChunk.toArray(inputs); - byte[][] outputBytesArr = ECChunk.toArray(outputs); - doDecode(inputBytesArr, erasedIndexes, outputBytesArr); - } else { - ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs); - ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs); - doDecode(inputBuffers, erasedIndexes, outputBuffers); + protected void checkParameters(Object[] inputs, int[] erasedIndexes, + Object[] outputs) { + if (inputs.length != getNumParityUnits() + getNumDataUnits()) { + throw new IllegalArgumentException("Invalid inputs length"); + } + + if (erasedIndexes.length != outputs.length) { + throw new IllegalArgumentException( + "erasedIndexes and outputs mismatch in length"); + } + + if (erasedIndexes.length > getNumParityUnits()) { + throw new IllegalArgumentException( + "Too many erased, not recoverable"); } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java index 4feaf39f28302..06e88bf287a37 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -31,23 +31,28 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder @Override public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { - assert (inputs.length == getNumDataUnits()); - assert (outputs.length == getNumParityUnits()); + checkParameters(inputs, outputs); - doEncode(inputs, outputs); + boolean hasArray = inputs[0].hasArray(); + if (hasArray) { + byte[][] newInputs = toArrays(inputs); + byte[][] newOutputs = toArrays(outputs); + doEncode(newInputs, newOutputs); + } else { + doEncode(inputs, outputs); + } } /** - * Perform the real encoding work using ByteBuffer - * @param inputs - * @param outputs + * Perform the real encoding work using direct ByteBuffer + * @param inputs Direct ByteBuffers expected + * @param outputs Direct ByteBuffers expected */ protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs); @Override public void encode(byte[][] inputs, byte[][] outputs) { - assert (inputs.length == getNumDataUnits()); - assert (outputs.length == getNumParityUnits()); + checkParameters(inputs, outputs); doEncode(inputs, outputs); } @@ -61,33 +66,22 @@ public void encode(byte[][] inputs, byte[][] outputs) { @Override public void encode(ECChunk[] inputs, ECChunk[] outputs) { - assert (inputs.length == getNumDataUnits()); - assert (outputs.length == getNumParityUnits()); - - doEncode(inputs, outputs); + ByteBuffer[] newInputs = ECChunk.toBuffers(inputs); + ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs); + encode(newInputs, newOutputs); } /** - * Perform the real encoding work using chunks. + * Check and validate decoding parameters, throw exception accordingly. * @param inputs * @param outputs */ - protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) { - /** - * Note callers may pass byte array, or ByteBuffer via ECChunk according - * to how ECChunk is created. Some implementations of coder use byte array - * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the - * better performance. - */ - if (inputs[0].getBuffer().hasArray()) { - byte[][] inputBytesArr = ECChunk.toArray(inputs); - byte[][] outputBytesArr = ECChunk.toArray(outputs); - doEncode(inputBytesArr, outputBytesArr); - } else { - ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs); - ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs); - doEncode(inputBuffers, outputBuffers); + protected void checkParameters(Object[] inputs, Object[] outputs) { + if (inputs.length != getNumDataUnits()) { + throw new IllegalArgumentException("Invalid inputs length"); + } + if (outputs.length != getNumParityUnits()) { + throw new IllegalArgumentException("Invalid outputs length"); } } - } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java index 9af5b6c0a9459..3fb211fd67808 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -64,13 +64,13 @@ public interface RawErasureCoder extends Configurable { public int getChunkSize(); /** - * Tell if native or off-heap buffer is preferred or not. It's for callers to - * decide how to allocate coding chunk buffers, either on heap or off heap. - * It will return false by default. + * Tell if direct buffer is preferred or not. It's for callers to + * decide how to allocate coding chunk buffers, using DirectByteBuffer or + * bytes array. It will return false by default. * @return true if native buffer is preferred for performance consideration, * otherwise false. */ - public boolean preferNativeBuffer(); + public boolean preferDirectBuffer(); /** * Should be called when release this coder. Good chance to release encoding diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java index 1358b7d0bc534..1807da729a20a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java @@ -31,24 +31,30 @@ public interface RawErasureDecoder extends RawErasureCoder { /** - * Decode with inputs and erasedIndexes, generates outputs - * @param inputs - * @param outputs + * Decode with inputs and erasedIndexes, generates outputs. + * @param inputs inputs to read data from + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs outputs to write into for data generated according to + * erasedIndexes */ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs); /** - * Decode with inputs and erasedIndexes, generates outputs - * @param inputs - * @param outputs + * Decode with inputs and erasedIndexes, generates outputs. + * @param inputs inputs to read data from + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs outputs to write into for data generated according to + * erasedIndexes */ public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs); /** - * Decode with inputs and erasedIndexes, generates outputs - * @param inputs - * @param outputs + * Decode with inputs and erasedIndexes, generates outputs. + * @param inputs inputs to read data from + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs outputs to write into for data generated according to + * erasedIndexes */ public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java index b6b163392e26f..2ea1b3d708441 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java @@ -27,17 +27,11 @@ public class XORRawDecoder extends AbstractRawErasureDecoder { @Override protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { - assert(erasedIndexes.length == outputs.length); - assert(erasedIndexes.length <= 1); + resetBuffer(outputs[0]); - int bufSize = inputs[0].remaining(); + int bufSize = getChunkSize(); int erasedIdx = erasedIndexes[0]; - // Set the output to zeros. - for (int j = 0; j < bufSize; j++) { - outputs[0].put(j, (byte) 0); - } - // Process the inputs. for (int i = 0; i < inputs.length; i++) { // Skip the erased location. @@ -52,19 +46,13 @@ protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, } @Override - protected void doDecode(byte[][] inputs, int[] erasedIndexes, - byte[][] outputs) { - assert(erasedIndexes.length == outputs.length); - assert(erasedIndexes.length <= 1); + protected void doDecode(byte[][] inputs, + int[] erasedIndexes, byte[][] outputs) { + resetBuffer(outputs[0]); - int bufSize = inputs[0].length; + int bufSize = getChunkSize(); int erasedIdx = erasedIndexes[0]; - // Set the output to zeros. - for (int j = 0; j < bufSize; j++) { - outputs[0][j] = 0; - } - // Process the inputs. for (int i = 0; i < inputs.length; i++) { // Skip the erased location. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java index dbfab5d2f65e5..116cb91ea3ffb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java @@ -26,8 +26,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder { @Override protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { - int bufSize = inputs[0].remaining(); + resetBuffer(outputs[0]); + int bufSize = getChunkSize(); // Get the first buffer's data. for (int j = 0; j < bufSize; j++) { outputs[0].put(j, inputs[0].get(j)); @@ -43,8 +44,9 @@ protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { @Override protected void doEncode(byte[][] inputs, byte[][] outputs) { - int bufSize = inputs[0].length; + resetBuffer(outputs[0]); + int bufSize = getChunkSize(); // Get the first buffer's data. for (int j = 0; j < bufSize; j++) { outputs[0][j] = inputs[0][j]; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index be1924c94f655..704b3f02bf071 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -75,8 +75,8 @@ protected Configuration getConf() { */ protected void compareAndVerify(ECChunk[] erasedChunks, ECChunk[] recoveredChunks) { - byte[][] erased = ECChunk.toArray(erasedChunks); - byte[][] recovered = ECChunk.toArray(recoveredChunks); + byte[][] erased = ECChunk.toArrays(erasedChunks); + byte[][] recovered = ECChunk.toArrays(recoveredChunks); boolean result = Arrays.deepEquals(erased, recovered); assertTrue("Decoding and comparing failed.", result); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java index 7d9d37a0a2546..3ae6a9359aad0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java @@ -56,14 +56,12 @@ public void testCodingDirectBufferWithConf_10x4_erasing_d0() { * This tests if the two configuration items work or not. */ Configuration conf = new Configuration(); - conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, - RSRawErasureCoderFactory.class.getCanonicalName()); - conf.setBoolean( - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false); + conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, RSRawErasureCoderFactory.class.getCanonicalName()); prepare(conf, 10, 4, new int[]{0}); testCoding(true); + testCoding(true); } @Test From b64f6745a45754dcf79c9c2626f3db7db2f33858 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 18 May 2015 10:13:03 -0700 Subject: [PATCH 119/212] HADOOP-11566. Add tests and fix for erasure coders to recover erased parity units. Contributed by Kai Zheng. --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 ++ .../apache/hadoop/io/erasurecode/ECChunk.java | 17 ++++++- .../coder/AbstractErasureDecoder.java | 13 +++-- .../hadoop/io/erasurecode/TestCoderBase.java | 37 ++++++++++---- .../coder/TestErasureCoderBase.java | 37 +++++++------- .../erasurecode/coder/TestRSErasureCoder.java | 48 +++++++++++-------- .../io/erasurecode/coder/TestXORCoder.java | 6 +-- .../erasurecode/rawcoder/TestRSRawCoder.java | 37 ++++++++------ .../rawcoder/TestRawCoderBase.java | 2 +- .../erasurecode/rawcoder/TestXORRawCoder.java | 11 ++++- 10 files changed, 134 insertions(+), 77 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index a152e317fd613..34dfc9e5ca010 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -48,3 +48,6 @@ HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang) HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang) + + HADOOP-11566. Add tests and fix for erasure coders to recover erased parity + units. (Kai Zheng via Zhe Zhang) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java index 436e13ef63f2d..69a8343ebe9f8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -58,8 +58,14 @@ public ByteBuffer getBuffer() { public static ByteBuffer[] toBuffers(ECChunk[] chunks) { ByteBuffer[] buffers = new ByteBuffer[chunks.length]; + ECChunk chunk; for (int i = 0; i < chunks.length; i++) { - buffers[i] = chunks[i].getBuffer(); + chunk = chunks[i]; + if (chunk == null) { + buffers[i] = null; + } else { + buffers[i] = chunk.getBuffer(); + } } return buffers; @@ -75,8 +81,15 @@ public static byte[][] toArrays(ECChunk[] chunks) { byte[][] bytesArr = new byte[chunks.length][]; ByteBuffer buffer; + ECChunk chunk; for (int i = 0; i < chunks.length; i++) { - buffer = chunks[i].getBuffer(); + chunk = chunks[i]; + if (chunk == null) { + bytesArr[i] = null; + continue; + } + + buffer = chunk.getBuffer(); if (buffer.hasArray()) { bytesArr[i] = buffer.array(); } else { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java index cd31294f80781..6437236355dc4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java @@ -60,16 +60,21 @@ protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { } /** - * Which blocks were erased ? We only care data blocks here. Sub-classes can - * override this behavior. + * Which blocks were erased ? * @param blockGroup * @return output blocks to recover */ protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { - ECBlock[] outputBlocks = new ECBlock[ - getNumErasedBlocks(blockGroup.getDataBlocks())]; + ECBlock[] outputBlocks = new ECBlock[getNumErasedBlocks(blockGroup)]; int idx = 0; + + for (int i = 0; i < getNumParityUnits(); i++) { + if (blockGroup.getParityBlocks()[i].isErased()) { + outputBlocks[idx++] = blockGroup.getParityBlocks()[i]; + } + } + for (int i = 0; i < getNumDataUnits(); i++) { if (blockGroup.getDataBlocks()[i].isErased()) { outputBlocks[idx++] = blockGroup.getDataBlocks()[i]; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 704b3f02bf071..769427d3a0437 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -37,10 +37,12 @@ public abstract class TestCoderBase { protected int numParityUnits; protected int chunkSize = 16 * 1024; - // Indexes of erased data units. Will also support test of erasing - // parity units + // Indexes of erased data units. protected int[] erasedDataIndexes = new int[] {0}; + // Indexes of erased parity units. + protected int[] erasedParityIndexes = new int[] {0}; + // Data buffers are either direct or on-heap, for performance the two cases // may go to different coding implementations. protected boolean usingDirectBuffer = true; @@ -52,12 +54,15 @@ public abstract class TestCoderBase { * @param erasedDataIndexes */ protected void prepare(Configuration conf, int numDataUnits, - int numParityUnits, int[] erasedDataIndexes) { + int numParityUnits, int[] erasedDataIndexes, + int[] erasedParityIndexes) { this.conf = conf; this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; this.erasedDataIndexes = erasedDataIndexes != null ? erasedDataIndexes : new int[] {0}; + this.erasedParityIndexes = erasedParityIndexes != null ? + erasedParityIndexes : new int[] {0}; } /** @@ -87,10 +92,15 @@ protected void compareAndVerify(ECChunk[] erasedChunks, * @return erased indexes altogether */ protected int[] getErasedIndexesForDecoding() { - int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length]; + int[] erasedIndexesForDecoding = + new int[erasedParityIndexes.length + erasedDataIndexes.length]; int idx = 0; + for (int i = 0; i < erasedParityIndexes.length; i++) { + erasedIndexesForDecoding[idx ++] = erasedParityIndexes[i]; + } + for (int i = 0; i < erasedDataIndexes.length; i++) { erasedIndexesForDecoding[idx ++] = erasedDataIndexes[i] + numParityUnits; } @@ -123,15 +133,25 @@ protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks, * Erase chunks to test the recovering of them. Before erasure clone them * first so could return them. * @param dataChunks + * @param parityChunks * @return clone of erased chunks */ - protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks) { - ECChunk[] toEraseChunks = new ECChunk[erasedDataIndexes.length]; + protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks, + ECChunk[] parityChunks) { + ECChunk[] toEraseChunks = new ECChunk[erasedParityIndexes.length + + erasedDataIndexes.length]; int idx = 0; + ECChunk chunk; + + for (int i = 0; i < erasedParityIndexes.length; i++) { + chunk = parityChunks[erasedParityIndexes[i]]; + toEraseChunks[idx ++] = cloneChunkWithData(chunk); + eraseDataFromChunk(chunk); + } for (int i = 0; i < erasedDataIndexes.length; i++) { - ECChunk chunk = dataChunks[erasedDataIndexes[i]]; + chunk = dataChunks[erasedDataIndexes[i]]; toEraseChunks[idx ++] = cloneChunkWithData(chunk); eraseDataFromChunk(chunk); } @@ -273,7 +293,8 @@ protected ECChunk[] prepareParityChunksForEncoding() { * @return */ protected ECChunk[] prepareOutputChunksForDecoding() { - ECChunk[] chunks = new ECChunk[erasedDataIndexes.length]; + ECChunk[] chunks = new ECChunk[erasedDataIndexes.length + + erasedParityIndexes.length]; for (int i = 0; i < chunks.length; i++) { chunks[i] = allocateOutputChunk(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index fdd0b50e282a9..f30323b590356 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -63,13 +63,15 @@ protected void testCoding(boolean usingDirectBuffer) { ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); // Backup all the source chunks for later recovering because some coders // may affect the source data. - TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[]) blockGroup.getDataBlocks()); + TestBlock[] clonedDataBlocks = + cloneBlocksWithData((TestBlock[]) blockGroup.getDataBlocks()); + TestBlock[] parityBlocks = (TestBlock[]) blockGroup.getParityBlocks(); ErasureCodingStep codingStep; codingStep = encoder.calculateCoding(blockGroup); performCodingStep(codingStep); // Erase specified sources but return copies of them for later comparing - TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks); + TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks, parityBlocks); // Decode blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks()); @@ -207,34 +209,27 @@ protected ECBlock generateDataBlock() { * @param dataBlocks * @return clone of erased dataBlocks */ - protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks) { - TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length]; + protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks, + TestBlock[] parityBlocks) { + TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length + + erasedParityIndexes.length]; int idx = 0; + TestBlock block; - for (int i = 0; i < erasedDataIndexes.length; i++) { - TestBlock block = dataBlocks[erasedDataIndexes[i]]; + for (int i = 0; i < erasedParityIndexes.length; i++) { + block = parityBlocks[erasedParityIndexes[i]]; toEraseBlocks[idx ++] = cloneBlockWithData(block); eraseDataFromBlock(block); } - return toEraseBlocks; - } - - /** - * Copy those data blocks that's to be erased for later comparing and - * verifying. - * @param dataBlocks - * @return - */ - protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) { - TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length]; - - for (int i = 0; i < erasedDataIndexes.length; ++i) { - copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]); + for (int i = 0; i < erasedDataIndexes.length; i++) { + block = dataBlocks[erasedDataIndexes[i]]; + toEraseBlocks[idx ++] = cloneBlockWithData(block); + eraseDataFromBlock(block); } - return copiedBlocks; + return toEraseBlocks; } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java index 3ae6a9359aad0..94f77db4fb305 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java @@ -40,8 +40,8 @@ public void setup() { } @Test - public void testCodingNoDirectBuffer_10x4_erasing_d0() { - prepare(null, 10, 4, new int[] {0}); + public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); /** * Doing twice to test if the coders can be repeatedly reused. This matters * as the underlying coding buffers are shared, which may have bugs. @@ -53,34 +53,41 @@ public void testCodingNoDirectBuffer_10x4_erasing_d0() { @Test public void testCodingDirectBufferWithConf_10x4_erasing_d0() { /** - * This tests if the two configuration items work or not. + * This tests if the configuration items work or not. */ Configuration conf = new Configuration(); - conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, RSRawErasureCoderFactory.class.getCanonicalName()); - - prepare(conf, 10, 4, new int[]{0}); + conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + RSRawErasureCoderFactory.class.getCanonicalName()); + prepare(conf, 10, 4, new int[]{0}, new int[0]); testCoding(true); testCoding(true); } - + + @Test + public void testCodingDirectBuffer_10x4_erasing_p1() { + prepare(null, 10, 4, new int[]{}, new int[]{1}); + testCoding(true); + testCoding(true); + } + @Test public void testCodingDirectBuffer_10x4_erasing_d2() { - prepare(null, 10, 4, new int[] {2}); + prepare(null, 10, 4, new int[] {2}, new int[] {}); testCoding(true); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasing_d0() { - prepare(null, 10, 4, new int[] {0}); + public void testCodingDirectBuffer_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); testCoding(true); testCoding(true); } @Test - public void testCodingBothBuffers_10x4_erasing_d0() { - prepare(null, 10, 4, new int[] {0}); + public void testCodingBothBuffers_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); /** * Doing in mixed buffer usage model to test if the coders can be repeatedly @@ -94,27 +101,26 @@ public void testCodingBothBuffers_10x4_erasing_d0() { } @Test - public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() { - prepare(null, 10, 4, new int[] {2, 4}); + public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() { + prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasing_d0_d1() { - prepare(null, 10, 4, new int[] {0, 1}); + public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() { + prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1}); testCoding(true); } @Test - public void testCodingNoDirectBuffer_3x3_erasing_d0() { - prepare(null, 3, 3, new int[] {0}); + public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() { + prepare(null, 3, 3, new int[] {0}, new int[] {0}); testCoding(false); } @Test - public void testCodingDirectBuffer_3x3_erasing_d0() { - prepare(null, 3, 3, new int[] {0}); + public void testCodingDirectBuffer_3x3_erasing_d0_p0() { + prepare(null, 3, 3, new int[] {0}, new int[] {0}); testCoding(true); } - } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java index 87aa656feac67..06e0087992871 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java @@ -36,8 +36,8 @@ public void setup() { } @Test - public void testCodingNoDirectBuffer_erasing_d0() { - prepare(null, 10, 1, new int[] {0}); + public void testCodingNoDirectBuffer_erasing_p0() { + prepare(null, 10, 1, new int[0], new int[] {0}); /** * Doing twice to test if the coders can be repeatedly reused. This matters @@ -49,7 +49,7 @@ public void testCodingNoDirectBuffer_erasing_d0() { @Test public void testCodingBothBuffers_erasing_d5() { - prepare(null, 10, 1, new int[]{5}); + prepare(null, 10, 1, new int[]{5}, new int[0]); /** * Doing in mixed buffer usage model to test if the coders can be repeatedly diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 9ba3e88ff1e30..84bad9231b79c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -32,8 +32,8 @@ public void setup() { } @Test - public void testCodingNoDirectBuffer_10x4_erasing_d0() { - prepare(null, 10, 4, new int[] {0}); + public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); /** * Doing twice to test if the coders can be repeatedly reused. This matters * as the underlying coding buffers are shared, which may have bugs. @@ -42,23 +42,30 @@ public void testCodingNoDirectBuffer_10x4_erasing_d0() { testCoding(false); } + @Test + public void testCodingDirectBuffer_10x4_erasing_p1() { + prepare(null, 10, 4, new int[] {}, new int[] {1}); + testCoding(true); + testCoding(true); + } + @Test public void testCodingDirectBuffer_10x4_erasing_d2() { - prepare(null, 10, 4, new int[] {2}); + prepare(null, 10, 4, new int[] {2}, new int[] {}); testCoding(true); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasing_d0() { - prepare(null, 10, 4, new int[] {0}); + public void testCodingDirectBuffer_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); testCoding(true); testCoding(true); } @Test - public void testCodingBothBuffers_10x4_erasing_d0() { - prepare(null, 10, 4, new int[] {0}); + public void testCodingBothBuffers_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); /** * Doing in mixed buffer usage model to test if the coders can be repeatedly @@ -72,26 +79,26 @@ public void testCodingBothBuffers_10x4_erasing_d0() { } @Test - public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() { - prepare(null, 10, 4, new int[] {2, 4}); + public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() { + prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}); testCoding(true); } @Test - public void testCodingDirectBuffer_10x4_erasing_d0_d1() { - prepare(null, 10, 4, new int[] {0, 1}); + public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() { + prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1}); testCoding(true); } @Test - public void testCodingNoDirectBuffer_3x3_erasing_d0() { - prepare(null, 3, 3, new int[] {0}); + public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() { + prepare(null, 3, 3, new int[] {0}, new int[] {0}); testCoding(false); } @Test - public void testCodingDirectBuffer_3x3_erasing_d0() { - prepare(null, 3, 3, new int[] {0}); + public void testCodingDirectBuffer_3x3_erasing_d0_p0() { + prepare(null, 3, 3, new int[] {0}, new int[] {0}); testCoding(true); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 7ba320a407987..8543c4d84f82b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -52,7 +52,7 @@ protected void testCoding(boolean usingDirectBuffer) { encoder.encode(dataChunks, parityChunks); // Backup and erase some chunks - ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks); + ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks, parityChunks); // Decode ECChunk[] inputChunks = prepareInputChunksForDecoding( diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java index 62ce4fbf36fef..b8912a9245b0f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java @@ -36,7 +36,7 @@ public void setup() { @Test public void testCodingNoDirectBuffer_erasing_d0() { - prepare(null, 10, 1, new int[] {0}); + prepare(null, 10, 1, new int[] {0}, new int[0]); /** * Doing twice to test if the coders can be repeatedly reused. This matters @@ -46,9 +46,16 @@ public void testCodingNoDirectBuffer_erasing_d0() { testCoding(false); } + @Test + public void testCodingDirectBuffer_erasing_p0() { + prepare(null, 10, 1, new int[0], new int[] {0}); + testCoding(true); + testCoding(true); + } + @Test public void testCodingBothBuffers_erasing_d5() { - prepare(null, 10, 1, new int[]{5}); + prepare(null, 10, 1, new int[]{5}, new int[0]); /** * Doing in mixed buffer usage model to test if the coders can be repeatedly From 343c0e76fcd95ac739ca7cd6742c9d617e19fc37 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 18 May 2015 10:14:54 -0700 Subject: [PATCH 120/212] HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure coder. Contributed by Kai Zheng. --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/io/erasurecode/ECChunk.java | 35 ++--- .../rawcoder/AbstractRawErasureCoder.java | 77 +++++----- .../rawcoder/AbstractRawErasureDecoder.java | 69 +++++++-- .../rawcoder/AbstractRawErasureEncoder.java | 66 +++++++-- .../io/erasurecode/rawcoder/RSRawDecoder.java | 22 +-- .../io/erasurecode/rawcoder/RSRawEncoder.java | 41 +++--- .../erasurecode/rawcoder/XORRawDecoder.java | 30 ++-- .../erasurecode/rawcoder/XORRawEncoder.java | 40 ++++-- .../rawcoder/util/GaloisField.java | 112 +++++++++++---- .../hadoop/io/erasurecode/TestCoderBase.java | 131 ++++++++++++++---- .../coder/TestErasureCoderBase.java | 21 ++- .../erasurecode/rawcoder/TestRSRawCoder.java | 12 +- .../rawcoder/TestRSRawCoderBase.java | 12 +- .../rawcoder/TestRawCoderBase.java | 57 +++++++- .../erasurecode/rawcoder/TestXORRawCoder.java | 19 +++ 16 files changed, 535 insertions(+), 212 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 34dfc9e5ca010..c799b4fed3349 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -51,3 +51,6 @@ HADOOP-11566. Add tests and fix for erasure coders to recover erased parity units. (Kai Zheng via Zhe Zhang) + + HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure + coder. (Kai Zheng via Zhe Zhang) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java index 69a8343ebe9f8..310c7382659b3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -72,34 +72,15 @@ public static ByteBuffer[] toBuffers(ECChunk[] chunks) { } /** - * Convert an array of this chunks to an array of byte array. - * Note the chunk buffers are not affected. - * @param chunks - * @return an array of byte array + * Convert to a bytes array, just for test usage. + * @return bytes array */ - public static byte[][] toArrays(ECChunk[] chunks) { - byte[][] bytesArr = new byte[chunks.length][]; - - ByteBuffer buffer; - ECChunk chunk; - for (int i = 0; i < chunks.length; i++) { - chunk = chunks[i]; - if (chunk == null) { - bytesArr[i] = null; - continue; - } - - buffer = chunk.getBuffer(); - if (buffer.hasArray()) { - bytesArr[i] = buffer.array(); - } else { - bytesArr[i] = new byte[buffer.remaining()]; - // Avoid affecting the original one - buffer.mark(); - buffer.get(bytesArr[i]); - buffer.reset(); - } - } + public byte[] toBytesArray() { + byte[] bytesArr = new byte[chunkBuffer.remaining()]; + // Avoid affecting the original one + chunkBuffer.mark(); + chunkBuffer.get(bytesArr); + chunkBuffer.reset(); return bytesArr; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index 2400313b42eb8..52689620a105f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configured; import java.nio.ByteBuffer; @@ -30,9 +31,6 @@ public abstract class AbstractRawErasureCoder extends Configured implements RawErasureCoder { - // Hope to reset coding buffers a little faster using it - private byte[] zeroChunkBytes; - private int numDataUnits; private int numParityUnits; private int chunkSize; @@ -43,8 +41,6 @@ public void initialize(int numDataUnits, int numParityUnits, this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; this.chunkSize = chunkSize; - - zeroChunkBytes = new byte[chunkSize]; // With ZERO by default } @Override @@ -73,55 +69,60 @@ public void release() { } /** - * Convert an array of heap ByteBuffers to an array of byte array. - * @param buffers - * @return an array of byte array + * Ensure output buffer filled with ZERO bytes fully in chunkSize. + * @param buffer a buffer ready to write chunk size bytes + * @return the buffer itself, with ZERO bytes written, the position and limit + * are not changed after the call */ - protected static byte[][] toArrays(ByteBuffer[] buffers) { - byte[][] bytesArr = new byte[buffers.length][]; - - ByteBuffer buffer; - for (int i = 0; i < buffers.length; i++) { - buffer = buffers[i]; - if (buffer == null) { - bytesArr[i] = null; - continue; - } - - if (buffer.hasArray()) { - bytesArr[i] = buffer.array(); - } else { - throw new IllegalArgumentException("Invalid ByteBuffer passed, " + - "expecting heap buffer"); - } + protected ByteBuffer resetOutputBuffer(ByteBuffer buffer) { + int pos = buffer.position(); + for (int i = pos; i < buffer.limit(); ++i) { + buffer.put((byte) 0); } + buffer.position(pos); - return bytesArr; + return buffer; } /** * Ensure the buffer (either input or output) ready to read or write with ZERO * bytes fully in chunkSize. - * @param buffer + * @param buffer bytes array buffer * @return the buffer itself */ - protected ByteBuffer resetBuffer(ByteBuffer buffer) { - buffer.clear(); - buffer.put(zeroChunkBytes); - buffer.position(0); + protected byte[] resetBuffer(byte[] buffer, int offset, int len) { + for (int i = offset; i < len; ++i) { + buffer[i] = (byte) 0; + } return buffer; } /** - * Ensure the buffer (either input or output) ready to read or write with ZERO - * bytes fully in chunkSize. - * @param buffer bytes array buffer - * @return the buffer itself + * Check and ensure the buffers are of the length specified by dataLen. + * @param buffers + * @param dataLen */ - protected byte[] resetBuffer(byte[] buffer) { - System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length); + protected void ensureLength(ByteBuffer[] buffers, int dataLen) { + for (int i = 0; i < buffers.length; ++i) { + if (buffers[i].remaining() != dataLen) { + throw new HadoopIllegalArgumentException( + "Invalid buffer, not of length " + dataLen); + } + } + } - return buffer; + /** + * Check and ensure the buffers are of the length specified by dataLen. + * @param buffers + * @param dataLen + */ + protected void ensureLength(byte[][] buffers, int dataLen) { + for (int i = 0; i < buffers.length; ++i) { + if (buffers[i].length != dataLen) { + throw new HadoopIllegalArgumentException( + "Invalid buffer not of length " + dataLen); + } + } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java index b247543fd326a..31f4fb858cd27 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.io.erasurecode.ECChunk; import java.nio.ByteBuffer; @@ -33,14 +34,43 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { checkParameters(inputs, erasedIndexes, outputs); + int dataLen = inputs[0].remaining(); + if (dataLen == 0) { + return; + } + ensureLength(inputs, dataLen); + ensureLength(outputs, dataLen); - boolean hasArray = inputs[0].hasArray(); - if (hasArray) { - byte[][] newInputs = toArrays(inputs); - byte[][] newOutputs = toArrays(outputs); - doDecode(newInputs, erasedIndexes, newOutputs); - } else { + boolean usingDirectBuffer = inputs[0].isDirect(); + if (usingDirectBuffer) { doDecode(inputs, erasedIndexes, outputs); + return; + } + + int[] inputOffsets = new int[inputs.length]; + int[] outputOffsets = new int[outputs.length]; + byte[][] newInputs = new byte[inputs.length][]; + byte[][] newOutputs = new byte[outputs.length][]; + + ByteBuffer buffer; + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + inputOffsets[i] = buffer.position(); + newInputs[i] = buffer.array(); + } + + for (int i = 0; i < outputs.length; ++i) { + buffer = outputs[i]; + outputOffsets[i] = buffer.position(); + newOutputs[i] = buffer.array(); + } + + doDecode(newInputs, inputOffsets, dataLen, + erasedIndexes, newOutputs, outputOffsets); + + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + buffer.position(inputOffsets[i] + dataLen); // dataLen bytes consumed } } @@ -56,18 +86,33 @@ protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, @Override public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) { checkParameters(inputs, erasedIndexes, outputs); + int dataLen = inputs[0].length; + if (dataLen == 0) { + return; + } + ensureLength(inputs, dataLen); + ensureLength(outputs, dataLen); + + int[] inputOffsets = new int[inputs.length]; // ALL ZERO + int[] outputOffsets = new int[outputs.length]; // ALL ZERO - doDecode(inputs, erasedIndexes, outputs); + doDecode(inputs, inputOffsets, dataLen, erasedIndexes, outputs, + outputOffsets); } /** - * Perform the real decoding using bytes array + * Perform the real decoding using bytes array, supporting offsets and + * lengths. * @param inputs + * @param inputOffsets + * @param dataLen * @param erasedIndexes * @param outputs + * @param outputOffsets */ - protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes, - byte[][] outputs); + protected abstract void doDecode(byte[][] inputs, int[] inputOffsets, + int dataLen, int[] erasedIndexes, + byte[][] outputs, int[] outputOffsets); @Override public void decode(ECChunk[] inputs, int[] erasedIndexes, @@ -91,12 +136,12 @@ protected void checkParameters(Object[] inputs, int[] erasedIndexes, } if (erasedIndexes.length != outputs.length) { - throw new IllegalArgumentException( + throw new HadoopIllegalArgumentException( "erasedIndexes and outputs mismatch in length"); } if (erasedIndexes.length > getNumParityUnits()) { - throw new IllegalArgumentException( + throw new HadoopIllegalArgumentException( "Too many erased, not recoverable"); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java index 06e88bf287a37..0ae54c5531651 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.io.erasurecode.ECChunk; import java.nio.ByteBuffer; @@ -32,14 +33,42 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder @Override public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { checkParameters(inputs, outputs); + int dataLen = inputs[0].remaining(); + if (dataLen == 0) { + return; + } + ensureLength(inputs, dataLen); + ensureLength(outputs, dataLen); - boolean hasArray = inputs[0].hasArray(); - if (hasArray) { - byte[][] newInputs = toArrays(inputs); - byte[][] newOutputs = toArrays(outputs); - doEncode(newInputs, newOutputs); - } else { + boolean usingDirectBuffer = inputs[0].isDirect(); + if (usingDirectBuffer) { doEncode(inputs, outputs); + return; + } + + int[] inputOffsets = new int[inputs.length]; + int[] outputOffsets = new int[outputs.length]; + byte[][] newInputs = new byte[inputs.length][]; + byte[][] newOutputs = new byte[outputs.length][]; + + ByteBuffer buffer; + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + inputOffsets[i] = buffer.position(); + newInputs[i] = buffer.array(); + } + + for (int i = 0; i < outputs.length; ++i) { + buffer = outputs[i]; + outputOffsets[i] = buffer.position(); + newOutputs[i] = buffer.array(); + } + + doEncode(newInputs, inputOffsets, dataLen, newOutputs, outputOffsets); + + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + buffer.position(buffer.position() + dataLen); // dataLen bytes consumed } } @@ -53,16 +82,31 @@ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { @Override public void encode(byte[][] inputs, byte[][] outputs) { checkParameters(inputs, outputs); + int dataLen = inputs[0].length; + if (dataLen == 0) { + return; + } + ensureLength(inputs, dataLen); + ensureLength(outputs, dataLen); + + int[] inputOffsets = new int[inputs.length]; // ALL ZERO + int[] outputOffsets = new int[outputs.length]; // ALL ZERO - doEncode(inputs, outputs); + doEncode(inputs, inputOffsets, dataLen, outputs, outputOffsets); } /** - * Perform the real encoding work using bytes array + * Perform the real encoding work using bytes array, supporting offsets + * and lengths. * @param inputs + * @param inputOffsets + * @param dataLen * @param outputs + * @param outputOffsets */ - protected abstract void doEncode(byte[][] inputs, byte[][] outputs); + protected abstract void doEncode(byte[][] inputs, int[] inputOffsets, + int dataLen, byte[][] outputs, + int[] outputOffsets); @Override public void encode(ECChunk[] inputs, ECChunk[] outputs) { @@ -78,10 +122,10 @@ public void encode(ECChunk[] inputs, ECChunk[] outputs) { */ protected void checkParameters(Object[] inputs, Object[] outputs) { if (inputs.length != getNumDataUnits()) { - throw new IllegalArgumentException("Invalid inputs length"); + throw new HadoopIllegalArgumentException("Invalid inputs length"); } if (outputs.length != getNumParityUnits()) { - throw new IllegalArgumentException("Invalid outputs length"); + throw new HadoopIllegalArgumentException("Invalid outputs length"); } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java index 24fa637426b7a..ff1162f494340 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java @@ -36,9 +36,9 @@ public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { super.initialize(numDataUnits, numParityUnits, chunkSize); assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); - this.errSignature = new int[getNumParityUnits()]; - this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(), - getNumParityUnits()); + this.errSignature = new int[numParityUnits]; + this.primitivePower = RSUtil.getPrimitivePower(numDataUnits, + numParityUnits); } @Override @@ -49,21 +49,21 @@ protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]); } - int dataLen = inputs[0].remaining(); - RSUtil.GF.solveVandermondeSystem(errSignature, outputs, - erasedIndexes.length, dataLen); + RSUtil.GF.solveVandermondeSystem(errSignature, + outputs, erasedIndexes.length); } @Override - protected void doDecode(byte[][] inputs, int[] erasedIndexes, - byte[][] outputs) { + protected void doDecode(byte[][] inputs, int[] inputOffsets, + int dataLen, int[] erasedIndexes, + byte[][] outputs, int[] outputOffsets) { for (int i = 0; i < erasedIndexes.length; i++) { errSignature[i] = primitivePower[erasedIndexes[i]]; - RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]); + RSUtil.GF.substitute(inputs, inputOffsets, dataLen, outputs[i], + outputOffsets[i], primitivePower[i]); } - int dataLen = inputs[0].length; - RSUtil.GF.solveVandermondeSystem(errSignature, outputs, + RSUtil.GF.solveVandermondeSystem(errSignature, outputs, outputOffsets, erasedIndexes.length, dataLen); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java index 7b501ceb453ca..9136331fbd6db 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java @@ -34,12 +34,12 @@ public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { super.initialize(numDataUnits, numParityUnits, chunkSize); assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); - int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(), - getNumParityUnits()); + int[] primitivePower = RSUtil.getPrimitivePower(numDataUnits, + numParityUnits); // compute generating polynomial int[] gen = {1}; int[] poly = new int[2]; - for (int i = 0; i < getNumParityUnits(); i++) { + for (int i = 0; i < numParityUnits; i++) { poly[0] = primitivePower[i]; poly[1] = 1; gen = RSUtil.GF.multiply(gen, poly); @@ -50,29 +50,30 @@ public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { @Override protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { - ByteBuffer[] data = new ByteBuffer[getNumDataUnits() + getNumParityUnits()]; - for (int i = 0; i < getNumParityUnits(); i++) { - data[i] = outputs[i]; - } - for (int i = 0; i < getNumDataUnits(); i++) { - data[i + getNumParityUnits()] = inputs[i]; - } + // parity units + data units + ByteBuffer[] all = new ByteBuffer[outputs.length + inputs.length]; + System.arraycopy(outputs, 0, all, 0, outputs.length); + System.arraycopy(inputs, 0, all, outputs.length, inputs.length); // Compute the remainder - RSUtil.GF.remainder(data, generatingPolynomial); + RSUtil.GF.remainder(all, generatingPolynomial); } @Override - protected void doEncode(byte[][] inputs, byte[][] outputs) { - byte[][] data = new byte[getNumDataUnits() + getNumParityUnits()][]; - for (int i = 0; i < getNumParityUnits(); i++) { - data[i] = outputs[i]; - } - for (int i = 0; i < getNumDataUnits(); i++) { - data[i + getNumParityUnits()] = inputs[i]; - } + protected void doEncode(byte[][] inputs, int[] inputOffsets, + int dataLen, byte[][] outputs, + int[] outputOffsets) { + // parity units + data units + byte[][] all = new byte[outputs.length + inputs.length][]; + System.arraycopy(outputs, 0, all, 0, outputs.length); + System.arraycopy(inputs, 0, all, outputs.length, inputs.length); + + int[] offsets = new int[inputOffsets.length + outputOffsets.length]; + System.arraycopy(outputOffsets, 0, offsets, 0, outputOffsets.length); + System.arraycopy(inputOffsets, 0, offsets, + outputOffsets.length, inputOffsets.length); // Compute the remainder - RSUtil.GF.remainder(data, generatingPolynomial); + RSUtil.GF.remainder(all, offsets, dataLen, generatingPolynomial); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java index 2ea1b3d708441..bf6e8942708c6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java @@ -21,47 +21,57 @@ /** * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID. + * + * XOR code is an important primitive code scheme in erasure coding and often + * used in advanced codes, like HitchHiker and LRC, though itself is rarely + * deployed independently. */ public class XORRawDecoder extends AbstractRawErasureDecoder { @Override protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { - resetBuffer(outputs[0]); + ByteBuffer output = outputs[0]; + resetOutputBuffer(output); - int bufSize = getChunkSize(); int erasedIdx = erasedIndexes[0]; // Process the inputs. + int iIdx, oIdx; for (int i = 0; i < inputs.length; i++) { // Skip the erased location. if (i == erasedIdx) { continue; } - for (int j = 0; j < bufSize; j++) { - outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j))); + for (iIdx = inputs[i].position(), oIdx = output.position(); + iIdx < inputs[i].limit(); + iIdx++, oIdx++) { + output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx))); } } } @Override - protected void doDecode(byte[][] inputs, - int[] erasedIndexes, byte[][] outputs) { - resetBuffer(outputs[0]); + protected void doDecode(byte[][] inputs, int[] inputOffsets, int dataLen, + int[] erasedIndexes, byte[][] outputs, + int[] outputOffsets) { + byte[] output = outputs[0]; + resetBuffer(output, outputOffsets[0], dataLen); - int bufSize = getChunkSize(); int erasedIdx = erasedIndexes[0]; // Process the inputs. + int iIdx, oIdx; for (int i = 0; i < inputs.length; i++) { // Skip the erased location. if (i == erasedIdx) { continue; } - for (int j = 0; j < bufSize; j++) { - outputs[0][j] ^= inputs[i][j]; + for (iIdx = inputOffsets[i], oIdx = outputOffsets[0]; + iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) { + output[oIdx] ^= inputs[i][iIdx]; } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java index 116cb91ea3ffb..feffbbf015825 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java @@ -21,43 +21,53 @@ /** * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID. + * + * XOR code is an important primitive code scheme in erasure coding and often + * used in advanced codes, like HitchHiker and LRC, though itself is rarely + * deployed independently. */ public class XORRawEncoder extends AbstractRawErasureEncoder { - @Override protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { - resetBuffer(outputs[0]); + ByteBuffer output = outputs[0]; + resetOutputBuffer(output); - int bufSize = getChunkSize(); // Get the first buffer's data. - for (int j = 0; j < bufSize; j++) { - outputs[0].put(j, inputs[0].get(j)); + int iIdx, oIdx; + for (iIdx = inputs[0].position(), oIdx = output.position(); + iIdx < inputs[0].limit(); iIdx++, oIdx++) { + output.put(oIdx, inputs[0].get(iIdx)); } // XOR with everything else. for (int i = 1; i < inputs.length; i++) { - for (int j = 0; j < bufSize; j++) { - outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j))); + for (iIdx = inputs[i].position(), oIdx = output.position(); + iIdx < inputs[i].limit(); + iIdx++, oIdx++) { + output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx))); } } } @Override - protected void doEncode(byte[][] inputs, byte[][] outputs) { - resetBuffer(outputs[0]); + protected void doEncode(byte[][] inputs, int[] inputOffsets, int dataLen, + byte[][] outputs, int[] outputOffsets) { + byte[] output = outputs[0]; + resetBuffer(output, outputOffsets[0], dataLen); - int bufSize = getChunkSize(); // Get the first buffer's data. - for (int j = 0; j < bufSize; j++) { - outputs[0][j] = inputs[0][j]; + int iIdx, oIdx; + for (iIdx = inputOffsets[0], oIdx = outputOffsets[0]; + iIdx < inputOffsets[0] + dataLen; iIdx++, oIdx++) { + output[oIdx] = inputs[0][iIdx]; } // XOR with everything else. for (int i = 1; i < inputs.length; i++) { - for (int j = 0; j < bufSize; j++) { - outputs[0][j] ^= inputs[i][j]; + for (iIdx = inputOffsets[i], oIdx = outputOffsets[0]; + iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) { + output[oIdx] ^= inputs[i][iIdx]; } } } - } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java index 77544c62235db..62b22c9fce309 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java @@ -235,26 +235,30 @@ public void solveVandermondeSystem(int[] x, int[] y, int len) { /** * A "bulk" version to the solving of Vandermonde System */ - public void solveVandermondeSystem(int[] x, byte[][] y, + public void solveVandermondeSystem(int[] x, byte[][] y, int[] outputOffsets, int len, int dataLen) { + int idx1, idx2; for (int i = 0; i < len - 1; i++) { for (int j = len - 1; j > i; j--) { - for (int k = 0; k < dataLen; k++) { - y[j][k] = (byte) (y[j][k] ^ mulTable[x[i]][y[j - 1][k] & + for (idx2 = outputOffsets[j-1], idx1 = outputOffsets[j]; + idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) { + y[j][idx1] = (byte) (y[j][idx1] ^ mulTable[x[i]][y[j - 1][idx2] & 0x000000FF]); } } } for (int i = len - 1; i >= 0; i--) { for (int j = i + 1; j < len; j++) { - for (int k = 0; k < dataLen; k++) { - y[j][k] = (byte) (divTable[y[j][k] & 0x000000FF][x[j] ^ + for (idx1 = outputOffsets[j]; + idx1 < outputOffsets[j] + dataLen; idx1++) { + y[j][idx1] = (byte) (divTable[y[j][idx1] & 0x000000FF][x[j] ^ x[j - i - 1]]); } } for (int j = i; j < len - 1; j++) { - for (int k = 0; k < dataLen; k++) { - y[j][k] = (byte) (y[j][k] ^ y[j + 1][k]); + for (idx2 = outputOffsets[j+1], idx1 = outputOffsets[j]; + idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) { + y[j][idx1] = (byte) (y[j][idx1] ^ y[j + 1][idx2]); } } } @@ -263,26 +267,34 @@ public void solveVandermondeSystem(int[] x, byte[][] y, /** * A "bulk" version of the solveVandermondeSystem, using ByteBuffer. */ - public void solveVandermondeSystem(int[] x, ByteBuffer[] y, - int len, int dataLen) { + public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) { + ByteBuffer p; + int idx1, idx2; for (int i = 0; i < len - 1; i++) { for (int j = len - 1; j > i; j--) { - for (int k = 0; k < dataLen; k++) { - y[j].put(k, (byte) (y[j].get(k) ^ mulTable[x[i]][y[j - 1].get(k) & + p = y[j]; + for (idx1 = p.position(), idx2 = y[j-1].position(); + idx1 < p.limit(); idx1++, idx2++) { + p.put(idx1, (byte) (p.get(idx1) ^ mulTable[x[i]][y[j-1].get(idx2) & 0x000000FF])); } } } + for (int i = len - 1; i >= 0; i--) { for (int j = i + 1; j < len; j++) { - for (int k = 0; k < dataLen; k++) { - y[j].put(k, (byte) (divTable[y[j].get(k) & 0x000000FF][x[j] ^ - x[j - i - 1]])); + p = y[j]; + for (idx1 = p.position(); idx1 < p.limit(); idx1++) { + p.put(idx1, (byte) (divTable[p.get(idx1) & + 0x000000FF][x[j] ^ x[j - i - 1]])); } } + for (int j = i; j < len - 1; j++) { - for (int k = 0; k < dataLen; k++) { - y[j].put(k, (byte) (y[j].get(k) ^ y[j + 1].get(k))); + p = y[j]; + for (idx1 = p.position(), idx2 = y[j+1].position(); + idx1 < p.limit(); idx1++, idx2++) { + p.put(idx1, (byte) (p.get(idx1) ^ y[j+1].get(idx2))); } } } @@ -393,6 +405,31 @@ public void substitute(byte[][] p, byte[] q, int x) { } } + /** + * A "bulk" version of the substitute. + * Tends to be 2X faster than the "int" substitute in a loop. + * + * @param p input polynomial + * @param offsets + * @param len + * @param q store the return result + * @param offset + * @param x input field + */ + public void substitute(byte[][] p, int[] offsets, + int len, byte[] q, int offset, int x) { + int y = 1, iIdx, oIdx; + for (int i = 0; i < p.length; i++) { + byte[] pi = p[i]; + for (iIdx = offsets[i], oIdx = offset; + iIdx < offsets[i] + len; iIdx++, oIdx++) { + int pij = pi[iIdx] & 0x000000FF; + q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]); + } + y = mulTable[x][y]; + } + } + /** * A "bulk" version of the substitute, using ByteBuffer. * Tends to be 2X faster than the "int" substitute in a loop. @@ -402,13 +439,13 @@ public void substitute(byte[][] p, byte[] q, int x) { * @param x input field */ public void substitute(ByteBuffer[] p, ByteBuffer q, int x) { - int y = 1; + int y = 1, iIdx, oIdx; for (int i = 0; i < p.length; i++) { ByteBuffer pi = p[i]; - int len = pi.remaining(); - for (int j = 0; j < len; j++) { - int pij = pi.get(j) & 0x000000FF; - q.put(j, (byte) (q.get(j) ^ mulTable[pij][y])); + for (iIdx = pi.position(), oIdx = q.position(); + iIdx < pi.limit(); iIdx++, oIdx++) { + int pij = pi.get(iIdx) & 0x000000FF; + q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y])); } y = mulTable[x][y]; } @@ -431,18 +468,43 @@ public void remainder(byte[][] dividend, int[] divisor) { } } + /** + * The "bulk" version of the remainder. + * Warning: This function will modify the "dividend" inputs. + */ + public void remainder(byte[][] dividend, int[] offsets, + int len, int[] divisor) { + int idx1, idx2; + for (int i = dividend.length - divisor.length; i >= 0; i--) { + for (int j = 0; j < divisor.length; j++) { + for (idx2 = offsets[j + i], idx1 = offsets[i + divisor.length - 1]; + idx1 < offsets[i + divisor.length - 1] + len; + idx1++, idx2++) { + int ratio = divTable[dividend[i + divisor.length - 1][idx1] & + 0x00FF][divisor[divisor.length - 1]]; + dividend[j + i][idx2] = (byte) ((dividend[j + i][idx2] & 0x00FF) ^ + mulTable[ratio][divisor[j]]); + } + } + } + } + /** * The "bulk" version of the remainder, using ByteBuffer. * Warning: This function will modify the "dividend" inputs. */ public void remainder(ByteBuffer[] dividend, int[] divisor) { + int idx1, idx2; + ByteBuffer b1, b2; for (int i = dividend.length - divisor.length; i >= 0; i--) { - int width = dividend[i].remaining(); for (int j = 0; j < divisor.length; j++) { - for (int k = 0; k < width; k++) { - int ratio = divTable[dividend[i + divisor.length - 1].get(k) & + b1 = dividend[i + divisor.length - 1]; + b2 = dividend[j + i]; + for (idx1 = b1.position(), idx2 = b2.position(); + idx1 < b1.limit(); idx1++, idx2++) { + int ratio = divTable[b1.get(idx1) & 0x00FF][divisor[divisor.length - 1]]; - dividend[j + i].put(k, (byte) ((dividend[j + i].get(k) & 0x00FF) ^ + b2.put(idx2, (byte) ((b2.get(idx2) & 0x00FF) ^ mulTable[ratio][divisor[j]])); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 769427d3a0437..cc3617cc436d9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -35,7 +35,12 @@ public abstract class TestCoderBase { private Configuration conf; protected int numDataUnits; protected int numParityUnits; - protected int chunkSize = 16 * 1024; + protected int baseChunkSize = 16 * 1024; + private int chunkSize = baseChunkSize; + + private byte[] zeroChunkBytes; + + private boolean startBufferWithZero = true; // Indexes of erased data units. protected int[] erasedDataIndexes = new int[] {0}; @@ -47,6 +52,15 @@ public abstract class TestCoderBase { // may go to different coding implementations. protected boolean usingDirectBuffer = true; + protected int getChunkSize() { + return chunkSize; + } + + protected void setChunkSize(int chunkSize) { + this.chunkSize = chunkSize; + this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default + } + /** * Prepare before running the case. * @param numDataUnits @@ -80,8 +94,8 @@ protected Configuration getConf() { */ protected void compareAndVerify(ECChunk[] erasedChunks, ECChunk[] recoveredChunks) { - byte[][] erased = ECChunk.toArrays(erasedChunks); - byte[][] recovered = ECChunk.toArrays(recoveredChunks); + byte[][] erased = toArrays(erasedChunks); + byte[][] recovered = toArrays(recoveredChunks); boolean result = Arrays.deepEquals(erased, recovered); assertTrue("Decoding and comparing failed.", result); } @@ -171,16 +185,19 @@ protected void eraseDataFromChunks(ECChunk[] chunks) { /** * Erase data from the specified chunk, putting ZERO bytes to the buffer. - * @param chunk + * @param chunk with a buffer ready to read at the current position */ protected void eraseDataFromChunk(ECChunk chunk) { ByteBuffer chunkBuffer = chunk.getBuffer(); - // erase the data - chunkBuffer.position(0); - for (int i = 0; i < chunkSize; i++) { - chunkBuffer.put((byte) 0); - } + // Erase the data at the position, and restore the buffer ready for reading + // same many bytes but all ZERO. + int pos = chunkBuffer.position(); + int len = chunkBuffer.remaining(); + chunkBuffer.put(zeroChunkBytes, 0, len); + // Back to readable again after data erased chunkBuffer.flip(); + chunkBuffer.position(pos); + chunkBuffer.limit(pos + len); } /** @@ -190,7 +207,7 @@ protected void eraseDataFromChunk(ECChunk chunk) { * @param chunks * @return */ - protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) { + protected ECChunk[] cloneChunksWithData(ECChunk[] chunks) { ECChunk[] results = new ECChunk[chunks.length]; for (int i = 0; i < chunks.length; i++) { results[i] = cloneChunkWithData(chunks[i]); @@ -206,22 +223,19 @@ protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) { * @param chunk * @return a new chunk */ - protected static ECChunk cloneChunkWithData(ECChunk chunk) { + protected ECChunk cloneChunkWithData(ECChunk chunk) { ByteBuffer srcBuffer = chunk.getBuffer(); - ByteBuffer destBuffer; byte[] bytesArr = new byte[srcBuffer.remaining()]; srcBuffer.mark(); - srcBuffer.get(bytesArr); + srcBuffer.get(bytesArr, 0, bytesArr.length); srcBuffer.reset(); - if (srcBuffer.hasArray()) { - destBuffer = ByteBuffer.wrap(bytesArr); - } else { - destBuffer = ByteBuffer.allocateDirect(srcBuffer.remaining()); - destBuffer.put(bytesArr); - destBuffer.flip(); - } + ByteBuffer destBuffer = allocateOutputBuffer(bytesArr.length); + int pos = destBuffer.position(); + destBuffer.put(bytesArr); + destBuffer.flip(); + destBuffer.position(pos); return new ECChunk(destBuffer); } @@ -231,18 +245,30 @@ protected static ECChunk cloneChunkWithData(ECChunk chunk) { * @return */ protected ECChunk allocateOutputChunk() { - ByteBuffer buffer = allocateOutputBuffer(); + ByteBuffer buffer = allocateOutputBuffer(chunkSize); return new ECChunk(buffer); } /** - * Allocate a buffer for output or writing. - * @return + * Allocate a buffer for output or writing. It can prepare for two kinds of + * data buffers: one with position as 0, the other with position > 0 + * @return a buffer ready to write chunkSize bytes from current position */ - protected ByteBuffer allocateOutputBuffer() { + protected ByteBuffer allocateOutputBuffer(int bufferLen) { + /** + * When startBufferWithZero, will prepare a buffer as:--------------- + * otherwise, the buffer will be like: ___TO--BE--WRITTEN___, + * and in the beginning, dummy data are prefixed, to simulate a buffer of + * position > 0. + */ + int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary + int allocLen = startOffset + bufferLen + startOffset; ByteBuffer buffer = usingDirectBuffer ? - ByteBuffer.allocateDirect(chunkSize) : ByteBuffer.allocate(chunkSize); + ByteBuffer.allocateDirect(allocLen) : ByteBuffer.allocate(allocLen); + buffer.limit(startOffset + bufferLen); + fillDummyData(buffer, startOffset); + startBufferWithZero = ! startBufferWithZero; return buffer; } @@ -265,15 +291,34 @@ protected ECChunk[] prepareDataChunksForEncoding() { * @return */ protected ECChunk generateDataChunk() { - ByteBuffer buffer = allocateOutputBuffer(); - for (int i = 0; i < chunkSize; i++) { - buffer.put((byte) RAND.nextInt(256)); - } + ByteBuffer buffer = allocateOutputBuffer(chunkSize); + int pos = buffer.position(); + buffer.put(generateData(chunkSize)); buffer.flip(); + buffer.position(pos); return new ECChunk(buffer); } + /** + * Fill len of dummy data in the buffer at the current position. + * @param buffer + * @param len + */ + protected void fillDummyData(ByteBuffer buffer, int len) { + byte[] dummy = new byte[len]; + RAND.nextBytes(dummy); + buffer.put(dummy); + } + + protected byte[] generateData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) RAND.nextInt(256); + } + return buffer; + } + /** * Prepare parity chunks for encoding, each chunk for each parity unit. * @return @@ -303,4 +348,32 @@ protected ECChunk[] prepareOutputChunksForDecoding() { return chunks; } + /** + * Convert an array of this chunks to an array of byte array. + * Note the chunk buffers are not affected. + * @param chunks + * @return an array of byte array + */ + protected byte[][] toArrays(ECChunk[] chunks) { + byte[][] bytesArr = new byte[chunks.length][]; + + for (int i = 0; i < chunks.length; i++) { + bytesArr[i] = chunks[i].toBytesArray(); + } + + return bytesArr; + } + + + /** + * Make some chunk messy or not correct any more + * @param chunks + */ + protected void corruptSomeChunk(ECChunk[] chunks) { + int idx = new Random().nextInt(chunks.length); + ByteBuffer buffer = chunks[idx].getBuffer(); + if (buffer.hasRemaining()) { + buffer.position(buffer.position() + 1); + } + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index f30323b590356..154ec1869944d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -59,6 +59,19 @@ protected void testCoding(boolean usingDirectBuffer) { this.usingDirectBuffer = usingDirectBuffer; prepareCoders(); + /** + * The following runs will use 3 different chunkSize for inputs and outputs, + * to verify the same encoder/decoder can process variable width of data. + */ + performTestCoding(baseChunkSize); + performTestCoding(baseChunkSize - 17); + performTestCoding(baseChunkSize + 16); + } + + private void performTestCoding(int chunkSize) { + setChunkSize(chunkSize); + + // Generate data and encode ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); // Backup all the source chunks for later recovering because some coders @@ -138,7 +151,7 @@ private ErasureCoder createEncoder() { throw new RuntimeException("Failed to create encoder", e); } - encoder.initialize(numDataUnits, numParityUnits, chunkSize); + encoder.initialize(numDataUnits, numParityUnits, getChunkSize()); encoder.setConf(getConf()); return encoder; } @@ -165,7 +178,7 @@ private ErasureCoder createDecoder() { throw new RuntimeException("Failed to create decoder", e); } - decoder.initialize(numDataUnits, numParityUnits, chunkSize); + decoder.initialize(numDataUnits, numParityUnits, getChunkSize()); decoder.setConf(getConf()); return decoder; } @@ -249,7 +262,7 @@ protected TestBlock allocateOutputBlock() { * @param blocks * @return */ - protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) { + protected TestBlock[] cloneBlocksWithData(TestBlock[] blocks) { TestBlock[] results = new TestBlock[blocks.length]; for (int i = 0; i < blocks.length; ++i) { results[i] = cloneBlockWithData(blocks[i]); @@ -263,7 +276,7 @@ protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) { * @param block * @return a new block */ - protected static TestBlock cloneBlockWithData(TestBlock block) { + protected TestBlock cloneBlockWithData(TestBlock block) { ECChunk[] newChunks = cloneChunksWithData(block.chunks); return new TestBlock(newChunks); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 84bad9231b79c..02b9eead3cc29 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -44,7 +44,7 @@ public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() { @Test public void testCodingDirectBuffer_10x4_erasing_p1() { - prepare(null, 10, 4, new int[] {}, new int[] {1}); + prepare(null, 10, 4, new int[0], new int[] {1}); testCoding(true); testCoding(true); } @@ -101,4 +101,14 @@ public void testCodingDirectBuffer_3x3_erasing_d0_p0() { prepare(null, 3, 3, new int[] {0}, new int[] {0}); testCoding(true); } + + @Test + public void testCodingNegative_10x4_erasing_d2_d4() { + prepare(null, 10, 4, new int[]{2, 4}, new int[0]); + + testCodingWithBadInput(true); + testCodingWithBadOutput(false); + testCodingWithBadInput(true); + testCodingWithBadOutput(false); + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java index f9e8a6baba11d..c06aded7b976d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java @@ -39,13 +39,11 @@ public abstract class TestRSRawCoderBase extends TestRawCoderBase { } @Override - protected ECChunk generateDataChunk() { - ByteBuffer buffer = allocateOutputBuffer(); - for (int i = 0; i < chunkSize; i++) { - buffer.put((byte) RAND.nextInt(symbolMax)); + protected byte[] generateData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < len; i++) { + buffer[i] = (byte) RAND.nextInt(symbolMax); } - buffer.flip(); - - return new ECChunk(buffer); + return buffer; } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 8543c4d84f82b..45823175cf12e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -19,6 +19,7 @@ import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.TestCoderBase; +import org.junit.Assert; /** * Raw coder test base with utilities. @@ -41,8 +42,57 @@ protected void testCoding(boolean usingDirectBuffer) { this.usingDirectBuffer = usingDirectBuffer; prepareCoders(); + /** + * The following runs will use 3 different chunkSize for inputs and outputs, + * to verify the same encoder/decoder can process variable width of data. + */ + performTestCoding(baseChunkSize, false, false); + performTestCoding(baseChunkSize - 17, false, false); + performTestCoding(baseChunkSize + 16, false, false); + } + + /** + * Similar to above, but perform negative cases using bad input for encoding. + * @param usingDirectBuffer + */ + protected void testCodingWithBadInput(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + prepareCoders(); + + try { + performTestCoding(baseChunkSize, true, false); + Assert.fail("Encoding test with bad input should fail"); + } catch (Exception e) { + // Expected + } + } + + /** + * Similar to above, but perform negative cases using bad output for decoding. + * @param usingDirectBuffer + */ + protected void testCodingWithBadOutput(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + prepareCoders(); + + try { + performTestCoding(baseChunkSize, false, true); + Assert.fail("Decoding test with bad output should fail"); + } catch (Exception e) { + // Expected + } + } + + private void performTestCoding(int chunkSize, + boolean useBadInput, boolean useBadOutput) { + setChunkSize(chunkSize); + // Generate data and encode ECChunk[] dataChunks = prepareDataChunksForEncoding(); + if (useBadInput) { + corruptSomeChunk(dataChunks); + } + ECChunk[] parityChunks = prepareParityChunksForEncoding(); // Backup all the source chunks for later recovering because some coders @@ -59,6 +109,9 @@ protected void testCoding(boolean usingDirectBuffer) { clonedDataChunks, parityChunks); ECChunk[] recoveredChunks = prepareOutputChunksForDecoding(); + if (useBadOutput) { + corruptSomeChunk(recoveredChunks); + } decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks); @@ -88,7 +141,7 @@ protected RawErasureEncoder createEncoder() { throw new RuntimeException("Failed to create encoder", e); } - encoder.initialize(numDataUnits, numParityUnits, chunkSize); + encoder.initialize(numDataUnits, numParityUnits, getChunkSize()); encoder.setConf(getConf()); return encoder; } @@ -105,7 +158,7 @@ protected RawErasureDecoder createDecoder() { throw new RuntimeException("Failed to create decoder", e); } - decoder.initialize(numDataUnits, numParityUnits, chunkSize); + decoder.initialize(numDataUnits, numParityUnits, getChunkSize()); decoder.setConf(getConf()); return decoder; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java index b8912a9245b0f..327174ef836cc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java @@ -49,6 +49,15 @@ public void testCodingNoDirectBuffer_erasing_d0() { @Test public void testCodingDirectBuffer_erasing_p0() { prepare(null, 10, 1, new int[0], new int[] {0}); + + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_erasing_d0() { + prepare(null, 10, 1, new int[] {0}, new int[0]); + testCoding(true); testCoding(true); } @@ -67,4 +76,14 @@ public void testCodingBothBuffers_erasing_d5() { testCoding(true); testCoding(false); } + + @Test + public void testCodingNegative_erasing_d5() { + prepare(null, 10, 1, new int[]{5}, new int[0]); + + testCodingWithBadInput(true); + testCodingWithBadOutput(false); + testCodingWithBadInput(true); + testCodingWithBadOutput(false); + } } From 6c310db15939bf5199894060607ec10ecfbe0877 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 18 May 2015 15:08:30 -0700 Subject: [PATCH 121/212] HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/util/StripedBlockUtil.java | 3 +- .../hdfs/TestDFSStripedInputStream.java | 72 +++++++++++-------- .../server/datanode/SimulatedFSDataset.java | 2 +- 4 files changed, 50 insertions(+), 30 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 333d85f0169a7..e016ba06790e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -217,3 +217,6 @@ assigning new tasks. (umamahesh) HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng) + + HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. + (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index c95f0b4e0a1b2..81c0c95d5ef93 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -379,7 +379,8 @@ public static AlignedStripe[] divideByteRangeIntoStripes ( int firstCellIdxInBG = (int) (start / cellSize); int lastCellIdxInBG = (int) (end / cellSize); int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len); - long firstCellOffsetInBlk = start % cellSize; + long firstCellOffsetInBlk = firstCellIdxInBG / dataBlkNum * cellSize + + start % cellSize; int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ? firstCellSize : (int) (end % cellSize) + 1; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 452cc2b9e81e8..9032d09d68696 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -38,6 +38,7 @@ import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -115,40 +116,55 @@ public void testPread() throws Exception { DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, NUM_STRIPE_PER_BLOCK, false); LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( - filePath.toString(), 0, BLOCK_GROUP_SIZE); + filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); + int fileLen = BLOCK_GROUP_SIZE * numBlocks; - assert lbs.get(0) instanceof LocatedStripedBlock; - LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); - for (int i = 0; i < DATA_BLK_NUM; i++) { - Block blk = new Block(bg.getBlock().getBlockId() + i, - NUM_STRIPE_PER_BLOCK * CELLSIZE, - bg.getBlock().getGenerationStamp()); - blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); - cluster.injectBlocks(i, Arrays.asList(blk), - bg.getBlock().getBlockPoolId()); - } - DFSStripedInputStream in = - new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema); - int readSize = BLOCK_GROUP_SIZE; - byte[] readBuffer = new byte[readSize]; - int ret = in.read(0, readBuffer, 0, readSize); + byte[] expected = new byte[fileLen]; + assertEquals(numBlocks, lbs.getLocatedBlocks().size()); + for (int bgIdx = 0; bgIdx < numBlocks; bgIdx++) { + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(bgIdx)); + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); + } - byte[] expected = new byte[readSize]; - /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ - for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { - for (int j = 0; j < DATA_BLK_NUM; j++) { - for (int k = 0; k < CELLSIZE; k++) { - int posInBlk = i * CELLSIZE + k; - int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; - expected[posInFile] = SimulatedFSDataset.simulatedByte( - new Block(bg.getBlock().getBlockId() + j), posInBlk); + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[bgIdx*BLOCK_GROUP_SIZE + posInFile] = + SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } } } } + DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), + filePath.toString(), false, schema); - assertEquals(readSize, ret); - assertArrayEquals(expected, readBuffer); + int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102, + CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102, + BLOCK_GROUP_SIZE - 102, BLOCK_GROUP_SIZE, BLOCK_GROUP_SIZE + 102, + fileLen - 1}; + for (int startOffset : startOffsets) { + startOffset = Math.max(0, Math.min(startOffset, fileLen - 1)); + int remaining = fileLen - startOffset; + byte[] buf = new byte[fileLen]; + int ret = in.read(startOffset, buf, 0, fileLen); + assertEquals(remaining, ret); + for (int i = 0; i < remaining; i++) { + Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + + "same", + expected[startOffset + i], buf[i]); + } + } + in.close(); } @Test 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 778dd2804c1e0..dd6b600e591b3 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 @@ -100,7 +100,7 @@ public static void setFactory(Configuration conf) { public static byte simulatedByte(Block b, long offsetInBlk) { byte firstByte = (byte) (b.getBlockId() & BYTE_MASK); - return (byte) ((firstByte + offsetInBlk) & BYTE_MASK); + return (byte) ((firstByte + offsetInBlk % 29) & BYTE_MASK); } public static final String CONFIG_PROPERTY_CAPACITY = From b008348dbf9bdd5070930be5d182116c5d370f6b Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 18 May 2015 19:06:34 -0700 Subject: [PATCH 122/212] HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN. Contributed by Yi Liu. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../server/blockmanagement/BlockManager.java | 54 ++++++++++++------- .../blockmanagement/DecommissionManager.java | 11 ++-- .../hdfs/server/namenode/NamenodeFsck.java | 2 +- 4 files changed, 43 insertions(+), 27 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index e016ba06790e8..154993051829a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -220,3 +220,6 @@ HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. (Walter Su via jing9) + + HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN. + (Yi Liu via jing9) 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 8b5144821eff2..d296aa8707327 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 @@ -43,7 +43,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; @@ -85,6 +84,7 @@ import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; import org.apache.hadoop.io.erasurecode.ECSchema; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; import org.apache.hadoop.net.Node; @@ -603,16 +603,7 @@ public int getDefaultStorageNum(BlockInfo block) { public short getMinStorageNum(BlockInfo block) { if (block.isStriped()) { - final BlockInfoStriped sblock = (BlockInfoStriped) block; - short dataBlockNum = sblock.getDataBlockNum(); - if (sblock.isComplete() || - sblock.getBlockUCState() == BlockUCState.COMMITTED) { - // if the sblock is committed/completed and its length is less than a - // full stripe, the minimum storage number needs to be adjusted - dataBlockNum = (short) Math.min(dataBlockNum, - (sblock.getNumBytes() - 1) / HdfsConstants.BLOCK_STRIPED_CELL_SIZE + 1); - } - return dataBlockNum; + return getStripedDataBlockNum(block); } else { return minReplication; } @@ -1258,7 +1249,7 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, return; } short expectedReplicas = - b.stored.getBlockCollection().getPreferredBlockReplication(); + getExpectedReplicaNum(b.stored.getBlockCollection(), b.stored); // Add replica to the data-node if it is not already there if (storageInfo != null) { @@ -1437,7 +1428,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { continue; } - requiredReplication = bc.getPreferredBlockReplication(); + requiredReplication = getExpectedReplicaNum(bc, block); // get a source data-node containingNodes = new ArrayList<>(); @@ -1537,7 +1528,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { rw.targets = null; continue; } - requiredReplication = bc.getPreferredBlockReplication(); + requiredReplication = getExpectedReplicaNum(bc, block); // do not schedule more if enough replicas is already pending NumberReplicas numReplicas = countNodes(block); @@ -2539,7 +2530,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( int reportedBlkIdx = BlockIdManager.getBlockIndex(reported); wrongSize = reported.getNumBytes() != getInternalBlockLength(stripedBlock.getNumBytes(), - HdfsConstants.BLOCK_STRIPED_CELL_SIZE, + BLOCK_STRIPED_CELL_SIZE, stripedBlock.getDataBlockNum(), reportedBlkIdx); } else { wrongSize = storedBlock.getNumBytes() != reported.getNumBytes(); @@ -2763,7 +2754,7 @@ private Block addStoredBlock(final BlockInfo block, } // handle underReplication/overReplication - short fileReplication = bc.getPreferredBlockReplication(); + short fileReplication = getExpectedReplicaNum(bc, storedBlock); if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) { neededReplications.remove(storedBlock, numCurrentReplica, num.decommissionedAndDecommissioning(), fileReplication); @@ -3003,7 +2994,7 @@ private MisReplicationResult processMisReplicatedBlock(BlockInfo block) { } // calculate current replication short expectedReplication = - block.getBlockCollection().getPreferredBlockReplication(); + getExpectedReplicaNum(block.getBlockCollection(), block); NumberReplicas num = countNodes(block); int numCurrentReplica = num.liveReplicas(); // add to under-replicated queue if need to be @@ -3638,8 +3629,8 @@ private void updateNeededReplications(final BlockInfo block, * process it as an over replicated block. */ public void checkReplication(BlockCollection bc) { - final short expected = bc.getPreferredBlockReplication(); for (BlockInfo block : bc.getBlocks()) { + short expected = getExpectedReplicaNum(bc, block); final NumberReplicas n = countNodes(block); if (isNeededReplication(block, expected, n.liveReplicas())) { neededReplications.add(block, n.liveReplicas(), @@ -3674,9 +3665,9 @@ public boolean checkBlocksProperlyReplicated( * @return 0 if the block is not found; * otherwise, return the replication factor of the block. */ - private int getReplication(Block block) { + private int getReplication(BlockInfo block) { final BlockCollection bc = blocksMap.getBlockCollection(block); - return bc == null? 0: bc.getPreferredBlockReplication(); + return bc == null? 0: getExpectedReplicaNum(bc, block); } @@ -3759,6 +3750,29 @@ boolean isNeededReplication(BlockInfo storedBlock, int expected, int current) { return current < expected || !blockHasEnoughRacks(storedBlock, expected); } + public short getExpectedReplicaNum(BlockCollection bc, BlockInfo block) { + if (block.isStriped()) { + return (short) (getStripedDataBlockNum(block) + + ((BlockInfoStriped) block).getParityBlockNum()); + } else { + return bc.getPreferredBlockReplication(); + } + } + + short getStripedDataBlockNum(BlockInfo block) { + assert block.isStriped(); + final BlockInfoStriped sblock = (BlockInfoStriped) block; + short dataBlockNum = sblock.getDataBlockNum(); + if (sblock.isComplete() || + sblock.getBlockUCState() == BlockUCState.COMMITTED) { + // if the sblock is committed/completed and its length is less than a + // full stripe, the minimum storage number needs to be adjusted + dataBlockNum = (short) Math.min(dataBlockNum, + (sblock.getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1); + } + return dataBlockNum; + } + public long getMissingBlocksCount() { // not locking return this.neededReplications.getCorruptBlockSize(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java index 37ce8e34bc57e..b1cc9bc3863c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java @@ -36,7 +36,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.util.CyclicIteration; @@ -251,7 +250,7 @@ private void setDecommissioned(DatanodeDescriptor dn) { */ private boolean isSufficient(BlockInfo block, BlockCollection bc, NumberReplicas numberReplicas) { - final int numExpected = bc.getPreferredBlockReplication(); + final int numExpected = blockManager.getExpectedReplicaNum(bc, block); final int numLive = numberReplicas.liveReplicas(); if (!blockManager.isNeededReplication(block, numExpected, numLive)) { // Block doesn't need replication. Skip. @@ -285,11 +284,11 @@ private boolean isSufficient(BlockInfo block, BlockCollection bc, return false; } - private static void logBlockReplicationInfo(Block block, BlockCollection bc, + private void logBlockReplicationInfo(BlockInfo block, BlockCollection bc, DatanodeDescriptor srcNode, NumberReplicas num, Iterable storages) { int curReplicas = num.liveReplicas(); - int curExpectedReplicas = bc.getPreferredBlockReplication(); + int curExpectedReplicas = blockManager.getExpectedReplicaNum(bc, block); StringBuilder nodeList = new StringBuilder(); for (DatanodeStorageInfo storage : storages) { final DatanodeDescriptor node = storage.getDatanodeDescriptor(); @@ -561,7 +560,7 @@ private void processBlocksForDecomInternal( // Schedule under-replicated blocks for replication if not already // pending if (blockManager.isNeededReplication(block, - bc.getPreferredBlockReplication(), liveReplicas)) { + blockManager.getExpectedReplicaNum(bc, block), liveReplicas)) { if (!blockManager.neededReplications.contains(block) && blockManager.pendingReplications.getNumReplicas(block) == 0 && namesystem.isPopulatingReplQueues()) { @@ -569,7 +568,7 @@ private void processBlocksForDecomInternal( blockManager.neededReplications.add(block, liveReplicas, num.decommissionedAndDecommissioning(), - bc.getPreferredBlockReplication()); + blockManager.getExpectedReplicaNum(bc, block)); } } 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 24a38e5e16fbf..fccef1795cfdf 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 @@ -256,7 +256,7 @@ public void blockIdCK(String blockId) { out.println("Block Id: " + blockId); out.println("Block belongs to: "+iNode.getFullPathName()); out.println("No. of Expected Replica: " + - bc.getPreferredBlockReplication()); + bm.getExpectedReplicaNum(bc, blockInfo)); out.println("No. of live Replica: " + numberReplicas.liveReplicas()); out.println("No. of excess Replica: " + numberReplicas.excessReplicas()); out.println("No. of stale Replica: " + From 7434c44b16765a49168c8f5cb01a3d0b817d158b Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 18 May 2015 22:55:27 -0700 Subject: [PATCH 123/212] HDFS-8320. Erasure coding: consolidate striping-related terminologies. Contributed by Zhe Zhang and Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/hdfs/DFSStripedInputStream.java | 46 +- .../erasurecode/ErasureCodingWorker.java | 4 +- .../hadoop/hdfs/util/StripedBlockUtil.java | 419 +++++++++--------- .../hadoop/hdfs/TestPlanReadPortions.java | 143 ------ .../hadoop/hdfs/TestRecoverStripedFile.java | 4 +- .../hdfs/util/TestStripedBlockUtil.java | 196 +++++++- 7 files changed, 400 insertions(+), 414 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 154993051829a..3170e9bc0fbce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -223,3 +223,5 @@ HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN. (Yi Liu via jing9) + + HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 8f15edad75127..744d5863afeaf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -23,19 +23,18 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.ByteBufferPool; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getStartOffsetsForInternalBlocks; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; @@ -65,30 +64,9 @@ import java.util.concurrent.Callable; import java.util.concurrent.Future; -/****************************************************************************** - * DFSStripedInputStream reads from striped block groups, illustrated below: - * - * | <- Striped Block Group -> | - * blk_0 blk_1 blk_2 <- A striped block group has - * | | | {@link #dataBlkNum} blocks - * v v v - * +------+ +------+ +------+ - * |cell_0| |cell_1| |cell_2| <- The logical read order should be - * +------+ +------+ +------+ cell_0, cell_1, ... - * |cell_3| |cell_4| |cell_5| - * +------+ +------+ +------+ - * |cell_6| |cell_7| |cell_8| - * +------+ +------+ +------+ - * |cell_9| - * +------+ <- A cell contains {@link #cellSize} bytes of data - * - * Three styles of read will eventually be supported: - * 1. Stateful read - * 2. pread without decode support - * This is implemented by calculating the portion of read from each block and - * issuing requests to each DataNode in parallel. - * 3. pread with decode support: TODO: will be supported after HDFS-7678 - *****************************************************************************/ +/** + * DFSStripedInputStream reads from striped block groups + */ public class DFSStripedInputStream extends DFSInputStream { private static class ReaderRetryPolicy { @@ -207,22 +185,24 @@ private synchronized void blockSeekTo(long target) throws IOException { currentLocatedBlock = targetBlockGroup; final long offsetIntoBlockGroup = getOffsetInBlockGroup(); - LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup( + LocatedBlock[] targetBlocks = parseStripedBlockGroup( targetBlockGroup, cellSize, dataBlkNum, parityBlkNum); - // The purpose is to get start offset into each block - ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, - offsetIntoBlockGroup, 0, 0); + // The purpose is to get start offset into each block. + long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema, + targetBlockGroup, offsetIntoBlockGroup); + Preconditions.checkNotNull(offsetsForInternalBlocks); final ReaderRetryPolicy retry = new ReaderRetryPolicy(); for (int i = 0; i < groupSize; i++) { LocatedBlock targetBlock = targetBlocks[i]; if (targetBlock != null) { + long offsetInBlock = offsetsForInternalBlocks[i] < 0 ? + 0 : offsetsForInternalBlocks[i]; DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); if (retval != null) { currentNodes[i] = retval.info; blockReaders[i] = getBlockReaderWithRetry(targetBlock, - readPortions[i].getStartOffsetInBlock(), - targetBlock.getBlockSize() - readPortions[i].getStartOffsetInBlock(), + offsetInBlock, targetBlock.getBlockSize() - offsetInBlock, retval.addr, retval.storageType, retval.info, target, retry); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 7b3c24d35a878..a1c0f722ce3a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -301,12 +301,12 @@ private class ReconstructAndTransferBlock implements Runnable { } private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) { - return StripedBlockUtil.constructStripedBlock(blockGroup, cellSize, + return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize, dataBlkNum, i); } private long getBlockLen(ExtendedBlock blockGroup, int i) { - return StripedBlockUtil.getStripedBlockLength(blockGroup.getNumBytes(), + return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(), cellSize, dataBlkNum, i); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 81c0c95d5ef93..2fa3fdf6ae8c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -41,7 +41,28 @@ import java.util.concurrent.TimeUnit; /** - * Utility class for analyzing striped block groups + * When accessing a file in striped layout, operations on logical byte ranges + * in the file need to be mapped to physical byte ranges on block files stored + * on DataNodes. This utility class facilities this mapping by defining and + * exposing a number of striping-related concepts. The most basic ones are + * illustrated in the following diagram. Unless otherwise specified, all + * range-related calculations are inclusive (the end offset of the previous + * range should be 1 byte lower than the start offset of the next one). + * + * | <---- Block Group ----> | <- Block Group: logical unit composing + * | | striped HDFS files. + * blk_0 blk_1 blk_2 <- Internal Blocks: each internal block + * | | | represents a physically stored local + * v v v block file + * +------+ +------+ +------+ + * |cell_0| |cell_1| |cell_2| <- {@link StripingCell} represents the + * +------+ +------+ +------+ logical order that a Block Group should + * |cell_3| |cell_4| |cell_5| be accessed: cell_0, cell_1, ... + * +------+ +------+ +------+ + * |cell_6| |cell_7| |cell_8| + * +------+ +------+ +------+ + * |cell_9| + * +------+ <- A cell contains cellSize bytes of data */ @InterfaceAudience.Private public class StripedBlockUtil { @@ -103,31 +124,6 @@ public static ExtendedBlock constructInternalBlock(ExtendedBlock blockGroup, cellSize, dataBlkNum, idxInBlockGroup)); return block; } - - /** - * This method creates an internal {@link ExtendedBlock} at the given index - * of a block group, for both data and parity block. - */ - public static ExtendedBlock constructStripedBlock(ExtendedBlock blockGroup, - int cellSize, int dataBlkNum, int idxInBlockGroup) { - ExtendedBlock block = new ExtendedBlock(blockGroup); - block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup); - block.setNumBytes(getStripedBlockLength(blockGroup.getNumBytes(), cellSize, - dataBlkNum, idxInBlockGroup)); - return block; - } - - /** - * Returns an internal block length at the given index of a block group, - * for both data and parity block. - */ - public static long getStripedBlockLength(long numBytes, int cellSize, - int dataBlkNum, int idxInBlockGroup) { - // parity block length is the same as the first striped block length. - return StripedBlockUtil.getInternalBlockLength( - numBytes, cellSize, dataBlkNum, - idxInBlockGroup < dataBlkNum ? idxInBlockGroup : 0); - } /** * Get the size of an internal block at the given index of a block group @@ -157,7 +153,7 @@ public static long getInternalBlockLength(long dataSize, return (numStripes - 1L)*cellSize + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i); } - + private static int lastCellSize(int size, int cellSize, int numDataBlocks, int i) { if (i < numDataBlocks) { @@ -183,60 +179,6 @@ public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum, + offsetInBlk % cellSize; // partial cell } - /** - * This method plans the read portion from each block in the stripe - * @param dataBlkNum The number of data blocks in the striping group - * @param cellSize The size of each striping cell - * @param startInBlk Starting offset in the striped block - * @param len Length of the read request - * @param bufOffset Initial offset in the result buffer - * @return array of {@link ReadPortion}, each representing the portion of I/O - * for an individual block in the group - */ - @VisibleForTesting - public static ReadPortion[] planReadPortions(final int dataBlkNum, - final int cellSize, final long startInBlk, final int len, int bufOffset) { - ReadPortion[] results = new ReadPortion[dataBlkNum]; - for (int i = 0; i < dataBlkNum; i++) { - results[i] = new ReadPortion(); - } - - // cellIdxInBlk is the index of the cell in the block - // E.g., cell_3 is the 2nd cell in blk_0 - int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum)); - - // blkIdxInGroup is the index of the block in the striped block group - // E.g., blk_2 is the 3rd block in the group - final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum); - results[blkIdxInGroup].setStartOffsetInBlock(cellSize * cellIdxInBlk + - startInBlk % cellSize); - boolean crossStripe = false; - for (int i = 1; i < dataBlkNum; i++) { - if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) { - cellIdxInBlk++; - crossStripe = true; - } - results[(blkIdxInGroup + i) % dataBlkNum].setStartOffsetInBlock( - cellSize * cellIdxInBlk); - } - - int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len); - results[blkIdxInGroup].offsetsInBuf.add(bufOffset); - results[blkIdxInGroup].lengths.add(firstCellLen); - results[blkIdxInGroup].addReadLength(firstCellLen); - - int i = (blkIdxInGroup + 1) % dataBlkNum; - for (int done = firstCellLen; done < len; done += cellSize) { - ReadPortion rp = results[i]; - rp.offsetsInBuf.add(done + bufOffset); - final int readLen = Math.min(len - done, cellSize); - rp.lengths.add(readLen); - rp.addReadLength(readLen); - i = (i + 1) % dataBlkNum; - } - return results; - } - /** * Get the next completed striped read task * @@ -360,84 +302,167 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf, } /** - * This method divides a requested byte range into an array of - * {@link AlignedStripe} - * + * This method divides a requested byte range into an array of inclusive + * {@link AlignedStripe}. + * @param ecSchema The codec schema for the file, which carries the numbers + * of data / parity blocks, as well as cell size + * @param blockGroup The striped block group + * @param rangeStartInBlockGroup The byte range's start offset in block group + * @param rangeEndInBlockGroup The byte range's end offset in block group + * @param buf Destination buffer of the read operation for the byte range + * @param offsetInBuf Start offset into the destination buffer * - * At most 5 stripes will be generated from each logical range - * TODO: cleanup and get rid of planReadPortions + * At most 5 stripes will be generated from each logical range, as + * demonstrated in the header of {@link AlignedStripe}. */ public static AlignedStripe[] divideByteRangeIntoStripes ( - ECSchema ecSchema, LocatedStripedBlock blockGroup, long start, long end, - byte[] buf, int offsetInBuf) { + ECSchema ecSchema, LocatedStripedBlock blockGroup, + long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf, + int offsetInBuf) { // TODO: change ECSchema naming to use cell size instead of chunk size // Step 0: analyze range and calculate basic parameters int cellSize = ecSchema.getChunkSize(); int dataBlkNum = ecSchema.getNumDataUnits(); - int len = (int) (end - start + 1); - int firstCellIdxInBG = (int) (start / cellSize); - int lastCellIdxInBG = (int) (end / cellSize); - int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len); - long firstCellOffsetInBlk = firstCellIdxInBG / dataBlkNum * cellSize + - start % cellSize; - int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ? - firstCellSize : (int) (end % cellSize) + 1; - - // Step 1: get the unmerged ranges on each internal block - // TODO: StripingCell should carry info on size and start offset (HDFS-8320) - VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, - firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk, - lastCellSize); - - // Step 2: merge into at most 5 stripes + + // Step 1: map the byte range to StripingCells + StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, blockGroup, + rangeStartInBlockGroup, rangeEndInBlockGroup); + + // Step 2: get the unmerged ranges on each internal block + VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cells); + + // Step 3: merge into at most 5 stripes AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); - // Step 3: calculate each chunk's position in destination buffer - calcualteChunkPositionsInBuf(ecSchema, blockGroup, buf, offsetInBuf, - firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk, - lastCellSize, stripes); + // Step 4: calculate each chunk's position in destination buffer + calcualteChunkPositionsInBuf(ecSchema, stripes, cells, buf, offsetInBuf); - // Step 4: prepare ALLZERO blocks + // Step 5: prepare ALLZERO blocks prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum); return stripes; } - private static VerticalRange[] getRangesForInternalBlocks (ECSchema ecSchema, - int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize, - long firstCellOffsetInBlk, int lastCellSize) { + /** + * Map the logical byte range to a set of inclusive {@link StripingCell} + * instances, each representing the overlap of the byte range to a cell + * used by {@link DFSStripedOutputStream} in encoding + */ + @VisibleForTesting + private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, + LocatedStripedBlock blockGroup, + long rangeStartInBlockGroup, long rangeEndInBlockGroup) { + Preconditions.checkArgument( + rangeStartInBlockGroup <= rangeEndInBlockGroup && + rangeEndInBlockGroup < blockGroup.getBlockSize()); int cellSize = ecSchema.getChunkSize(); - int dataBlkNum = ecSchema.getNumDataUnits(); + int len = (int) (rangeEndInBlockGroup - rangeStartInBlockGroup + 1); + int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); + int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize); + int numCells = lastCellIdxInBG - firstCellIdxInBG + 1; + StripingCell[] cells = new StripingCell[numCells]; + cells[0] = new StripingCell(ecSchema, firstCellIdxInBG); + cells[numCells - 1] = new StripingCell(ecSchema, lastCellIdxInBG); + + cells[0].offset = (int) (rangeStartInBlockGroup % cellSize); + cells[0].size = + Math.min(cellSize - (int) (rangeStartInBlockGroup % cellSize), len); + if (lastCellIdxInBG != firstCellIdxInBG) { + cells[numCells - 1].size = (int) (rangeEndInBlockGroup % cellSize) + 1; + } + + for (int i = 1; i < numCells - 1; i++) { + cells[i] = new StripingCell(ecSchema, i + firstCellIdxInBG); + } + + return cells; + } + /** + * Given a logical start offset in a block group, calculate the physical + * start offset into each stored internal block. + */ + public static long[] getStartOffsetsForInternalBlocks( + ECSchema ecSchema, LocatedStripedBlock blockGroup, + long rangeStartInBlockGroup) { + Preconditions.checkArgument( + rangeStartInBlockGroup < blockGroup.getBlockSize()); + int dataBlkNum = ecSchema.getNumDataUnits(); + int parityBlkNum = ecSchema.getNumParityUnits(); + int cellSize = ecSchema.getChunkSize(); + long[] startOffsets = new long[dataBlkNum + parityBlkNum]; + Arrays.fill(startOffsets, -1L); + int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG); - StripingCell lastCell = new StripingCell(ecSchema, lastCellIdxInBG); + firstCell.offset = (int) (rangeStartInBlockGroup % cellSize); + startOffsets[firstCell.idxInStripe] = + firstCell.idxInInternalBlk * cellSize + firstCell.offset; + long earliestStart = startOffsets[firstCell.idxInStripe]; + for (int i = 1; i < dataBlkNum; i++) { + int idx = firstCellIdxInBG + i; + if (idx * cellSize >= blockGroup.getBlockSize()) { + break; + } + StripingCell cell = new StripingCell(ecSchema, idx); + startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize; + if (startOffsets[cell.idxInStripe] < earliestStart) { + earliestStart = startOffsets[cell.idxInStripe]; + } + } + for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { + startOffsets[i] = earliestStart; + } + return startOffsets; + } - VerticalRange ranges[] = new VerticalRange[dataBlkNum]; - ranges[firstCell.idxInStripe] = - new VerticalRange(firstCellOffsetInBlk, firstCellSize); - for (int i = firstCellIdxInBG + 1; i < lastCellIdxInBG; i++) { + /** + * Given a logical byte range, mapped to each {@link StripingCell}, calculate + * the physical byte range (inclusive) on each stored internal block. + */ + @VisibleForTesting + private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema, + StripingCell[] cells) { + int cellSize = ecSchema.getChunkSize(); + int dataBlkNum = ecSchema.getNumDataUnits(); + int parityBlkNum = ecSchema.getNumParityUnits(); + + VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum]; + + long earliestStart = Long.MAX_VALUE; + long latestEnd = -1; + for (StripingCell cell : cells) { // iterate through all cells and update the list of StripeRanges - StripingCell cell = new StripingCell(ecSchema, i); if (ranges[cell.idxInStripe] == null) { ranges[cell.idxInStripe] = new VerticalRange( - cell.idxInInternalBlk * cellSize, cellSize); + cell.idxInInternalBlk * cellSize + cell.offset, cell.size); } else { - ranges[cell.idxInStripe].spanInBlock += cellSize; + ranges[cell.idxInStripe].spanInBlock += cell.size; + } + VerticalRange range = ranges[cell.idxInStripe]; + if (range.offsetInBlock < earliestStart) { + earliestStart = range.offsetInBlock; + } + if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) { + latestEnd = range.offsetInBlock + range.spanInBlock - 1; } } - if (ranges[lastCell.idxInStripe] == null) { - ranges[lastCell.idxInStripe] = new VerticalRange( - lastCell.idxInInternalBlk * cellSize, lastCellSize); - } else if (lastCell.idxInBlkGroup != firstCell.idxInBlkGroup) { - ranges[lastCell.idxInStripe].spanInBlock += lastCellSize; + + // Each parity block should be fetched at maximum range of all data blocks + for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { + ranges[i] = new VerticalRange(earliestStart, + latestEnd - earliestStart + 1); } return ranges; } - private static AlignedStripe[] mergeRangesForInternalBlocks(ECSchema ecSchema, - VerticalRange[] ranges) { + /** + * Merge byte ranges on each internal block into a set of inclusive + * {@link AlignedStripe} instances. + */ + private static AlignedStripe[] mergeRangesForInternalBlocks( + ECSchema ecSchema, VerticalRange[] ranges) { int dataBlkNum = ecSchema.getNumDataUnits(); int parityBlkNum = ecSchema.getNumParityUnits(); List stripes = new ArrayList<>(); @@ -461,12 +486,8 @@ private static AlignedStripe[] mergeRangesForInternalBlocks(ECSchema ecSchema, } private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, - LocatedStripedBlock blockGroup, byte[] buf, int offsetInBuf, - int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize, - long firstCellOffsetInBlk, int lastCellSize, AlignedStripe[] stripes) { - int cellSize = ecSchema.getChunkSize(); - int dataBlkNum = ecSchema.getNumDataUnits(); - // Step 3: calculate each chunk's position in destination buffer + AlignedStripe[] stripes, StripingCell[] cells, byte[] buf, + int offsetInBuf) { /** * | <--------------- AlignedStripe --------------->| * @@ -484,20 +505,11 @@ private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, * * Cell indexing convention defined in {@link StripingCell} */ + int cellSize = ecSchema.getChunkSize(); int done = 0; - for (int i = firstCellIdxInBG; i <= lastCellIdxInBG; i++) { - StripingCell cell = new StripingCell(ecSchema, i); - long cellStart = i == firstCellIdxInBG ? - firstCellOffsetInBlk : cell.idxInInternalBlk * cellSize; - int cellLen; - if (i == firstCellIdxInBG) { - cellLen = firstCellSize; - } else if (i == lastCellIdxInBG) { - cellLen = lastCellSize; - } else { - cellLen = cellSize; - } - long cellEnd = cellStart + cellLen - 1; + for (StripingCell cell : cells) { + long cellStart = cell.idxInInternalBlk * cellSize + cell.offset; + long cellEnd = cellStart + cell.size - 1; for (AlignedStripe s : stripes) { long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1; long overlapStart = Math.max(cellStart, s.getOffsetInBlock()); @@ -514,10 +526,14 @@ private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, add((int)(offsetInBuf + done + overlapStart - cellStart)); s.chunks[cell.idxInStripe].lengthsInBuf.add(overLapLen); } - done += cellLen; + done += cell.size; } } + /** + * If a {@link StripingChunk} maps to a byte range beyond an internal block's + * size, the chunk should be treated as zero bytes in decoding. + */ private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, byte[] buf, AlignedStripe[] stripes, int cellSize, int dataBlkNum) { for (AlignedStripe s : stripes) { @@ -534,51 +550,13 @@ private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, } /** - * This class represents the portion of I/O associated with each block in the - * striped block group. - * TODO: consolidate ReadPortion with AlignedStripe - */ - public static class ReadPortion { - private long startOffsetInBlock = 0; - private int readLength = 0; - public final List offsetsInBuf = new ArrayList<>(); - public final List lengths = new ArrayList<>(); - - public int[] getOffsets() { - int[] offsets = new int[offsetsInBuf.size()]; - for (int i = 0; i < offsets.length; i++) { - offsets[i] = offsetsInBuf.get(i); - } - return offsets; - } - - public int[] getLengths() { - int[] lens = new int[this.lengths.size()]; - for (int i = 0; i < lens.length; i++) { - lens[i] = this.lengths.get(i); - } - return lens; - } - - public long getStartOffsetInBlock() { - return startOffsetInBlock; - } - - public int getReadLength() { - return readLength; - } - - public void setStartOffsetInBlock(long startOffsetInBlock) { - this.startOffsetInBlock = startOffsetInBlock; - } - - void addReadLength(int extraLength) { - this.readLength += extraLength; - } - } - - /** - * The unit of encoding used in {@link DFSStripedOutputStream} + * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This + * size impacts how a logical offset in the file or block group translates + * to physical byte offset in a stored internal block. The StripingCell util + * class facilitates this calculation. Each StripingCell is inclusive with + * its start and end offsets -- e.g., the end logical offset of cell_0_0_0 + * should be 1 byte lower than the start logical offset of cell_1_0_1. + * * | <------- Striped Block Group -------> | * blk_0 blk_1 blk_2 * | | | @@ -586,43 +564,57 @@ void addReadLength(int extraLength) { * +----------+ +----------+ +----------+ * |cell_0_0_0| |cell_1_0_1| |cell_2_0_2| * +----------+ +----------+ +----------+ - * |cell_3_1_0| |cell_4_1_1| |cell_5_1_2| <- {@link idxInBlkGroup} = 5 - * +----------+ +----------+ +----------+ {@link idxInInternalBlk} = 1 - * {@link idxInStripe} = 2 + * |cell_3_1_0| |cell_4_1_1| |cell_5_1_2| <- {@link #idxInBlkGroup} = 5 + * +----------+ +----------+ +----------+ {@link #idxInInternalBlk} = 1 + * {@link #idxInStripe} = 2 * A StripingCell is a special instance of {@link StripingChunk} whose offset * and size align with the cell used when writing data. * TODO: consider parity cells */ - public static class StripingCell { + @VisibleForTesting + static class StripingCell { public final ECSchema schema; /** Logical order in a block group, used when doing I/O to a block group */ - public final int idxInBlkGroup; - public final int idxInInternalBlk; - public final int idxInStripe; + final int idxInBlkGroup; + final int idxInInternalBlk; + final int idxInStripe; + /** + * When a logical byte range is mapped to a set of cells, it might + * partially overlap with the first and last cells. This field and the + * {@link #size} variable represent the start offset and size of the + * overlap. + */ + int offset; + int size; - public StripingCell(ECSchema ecSchema, int idxInBlkGroup) { + StripingCell(ECSchema ecSchema, int idxInBlkGroup) { this.schema = ecSchema; this.idxInBlkGroup = idxInBlkGroup; this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits(); this.idxInStripe = idxInBlkGroup - this.idxInInternalBlk * ecSchema.getNumDataUnits(); + this.offset = 0; + this.size = ecSchema.getChunkSize(); } - public StripingCell(ECSchema ecSchema, int idxInInternalBlk, + StripingCell(ECSchema ecSchema, int idxInInternalBlk, int idxInStripe) { this.schema = ecSchema; this.idxInInternalBlk = idxInInternalBlk; this.idxInStripe = idxInStripe; this.idxInBlkGroup = idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe; + this.offset = 0; + this.size = ecSchema.getChunkSize(); } } /** * Given a requested byte range on a striped block group, an AlignedStripe - * represents a {@link VerticalRange} that is aligned with both the byte range - * and boundaries of all internal blocks. As illustrated in the diagram, any - * given byte range on a block group leads to 1~5 AlignedStripe's. + * represents an inclusive {@link VerticalRange} that is aligned with both + * the byte range and boundaries of all internal blocks. As illustrated in + * the diagram, any given byte range on a block group leads to 1~5 + * AlignedStripe's. * * |<-------- Striped Block Group -------->| * blk_0 blk_1 blk_2 blk_3 blk_4 @@ -648,6 +640,7 @@ public StripingCell(ECSchema ecSchema, int idxInInternalBlk, * * The coverage of an AlignedStripe on an internal block is represented as a * {@link StripingChunk}. + * * To simplify the logic of reading a logical byte range from a block group, * a StripingChunk is either completely in the requested byte range or * completely outside the requested byte range. @@ -692,19 +685,19 @@ public String toString() { /** * A simple utility class representing an arbitrary vertical inclusive range - * starting at {@link offsetInBlock} and lasting for {@link length} bytes in - * an internal block. Note that VerticalRange doesn't necessarily align with - * {@link StripingCell}. + * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock} + * bytes in an internal block. Note that VerticalRange doesn't necessarily + * align with {@link StripingCell}. * * |<- Striped Block Group ->| * blk_0 * | * v * +-----+ - * |~~~~~| <-- {@link offsetInBlock} + * |~~~~~| <-- {@link #offsetInBlock} * | | ^ * | | | - * | | | {@link spanInBlock} + * | | | {@link #spanInBlock} * | | v * |~~~~~| --- * | | @@ -743,9 +736,9 @@ public boolean include(long pos) { * +---------+ +---------+ | +----+ +----+ * <----------- data blocks ------------> | <--- parity ---> * - * The class also carries {@link buf}, {@link offsetsInBuf}, and - * {@link lengthsInBuf} to define how read task for this chunk should deliver - * the returned data. + * The class also carries {@link #buf}, {@link #offsetsInBuf}, and + * {@link #lengthsInBuf} to define how read task for this chunk should + * deliver the returned data. */ public static class StripingChunk { /** Chunk has been successfully fetched */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java deleted file mode 100644 index 75d05879f213e..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java +++ /dev/null @@ -1,143 +0,0 @@ -/** - * 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.junit.Test; - -import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion; -import static org.junit.Assert.*; - -public class TestPlanReadPortions { - - // We only support this as num of data blocks. It might be good enough for now - // for the purpose, even not flexible yet for any number in a schema. - private final short GROUP_SIZE = 3; - private final int CELLSIZE = 128 * 1024; - - private void testPlanReadPortions(int startInBlk, int length, - int bufferOffset, int[] readLengths, int[] offsetsInBlock, - int[][] bufferOffsets, int[][] bufferLengths) { - ReadPortion[] results = StripedBlockUtil.planReadPortions(GROUP_SIZE, - CELLSIZE, startInBlk, length, bufferOffset); - assertEquals(GROUP_SIZE, results.length); - - for (int i = 0; i < GROUP_SIZE; i++) { - assertEquals(readLengths[i], results[i].getReadLength()); - assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock()); - final int[] bOffsets = results[i].getOffsets(); - assertArrayEquals(bufferOffsets[i], bOffsets); - final int[] bLengths = results[i].getLengths(); - assertArrayEquals(bufferLengths[i], bLengths); - } - } - - /** - * Test {@link StripedBlockUtil#planReadPortions} - */ - @Test - public void testPlanReadPortions() { - /** - * start block offset is 0, read cellSize - 10 - */ - testPlanReadPortions(0, CELLSIZE - 10, 0, - new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0}, - new int[][]{new int[]{0}, new int[]{}, new int[]{}}, - new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}}); - - /** - * start block offset is 0, read 3 * cellSize - */ - testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0, - new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0}, - new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}}, - new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}}); - - /** - * start block offset is 0, read cellSize + 10 - */ - testPlanReadPortions(0, CELLSIZE + 10, 0, - new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0}, - new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}}, - new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}}); - - /** - * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100 - */ - testPlanReadPortions(0, 5 * CELLSIZE + 10, 100, - new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0}, - new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE}, - new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4}, - new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}}, - new int[][]{new int[]{CELLSIZE, CELLSIZE}, - new int[]{CELLSIZE, CELLSIZE}, - new int[]{CELLSIZE, 10}}); - - /** - * start block offset is 2, read 3 * cellSize - */ - testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100, - new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, - new int[]{2, 0, 0}, - new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2}, - new int[]{100 + CELLSIZE - 2}, - new int[]{100 + CELLSIZE * 2 - 2}}, - new int[][]{new int[]{CELLSIZE - 2, 2}, - new int[]{CELLSIZE}, - new int[]{CELLSIZE}}); - - /** - * start block offset is 2, read 3 * cellSize + 10 - */ - testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0, - new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE}, - new int[]{2, 0, 0}, - new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2}, - new int[]{CELLSIZE - 2}, - new int[]{CELLSIZE * 2 - 2}}, - new int[][]{new int[]{CELLSIZE - 2, 12}, - new int[]{CELLSIZE}, - new int[]{CELLSIZE}}); - - /** - * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10 - */ - testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0, - new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2}, - new int[]{CELLSIZE, CELLSIZE - 1, 0}, - new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1}, - new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}, - new int[]{1, 3 * CELLSIZE + 1}}, - new int[][]{new int[]{CELLSIZE, CELLSIZE}, - new int[]{1, CELLSIZE, 9}, - new int[]{CELLSIZE, CELLSIZE}}); - - /** - * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10 - */ - testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0, - new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1}, - new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1}, - new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1}, - new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1}, - new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}}, - new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, - new int[]{CELLSIZE, CELLSIZE, 9}, - new int[]{1, CELLSIZE, CELLSIZE}}); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index b4f05d46b9a89..dfdcee29f6ca5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -189,13 +189,13 @@ private void assertFileBlocksRecovery(String fileName, int fileLen, deadDnIndices[i] = dnMap.get(dataDNs[i]); // Check the block replica file on deadDn before it dead. - blocks[i] = StripedBlockUtil.constructStripedBlock( + blocks[i] = StripedBlockUtil.constructInternalBlock( lastBlock.getBlock(), cellSize, dataBlkNum, indices[toDead[i]]); replicas[i] = cluster.getBlockFile(deadDnIndices[i], blocks[i]); metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]); // the block replica on the datanode should be the same as expected assertEquals(replicas[i].length(), - StripedBlockUtil.getStripedBlockLength( + StripedBlockUtil.getInternalBlockLength( lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]])); assertTrue(metadatas[i].getName(). endsWith(blocks[i].getGenerationStamp() + ".meta")); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java index ec0b1bbdae6e2..6f29d6904831d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.util; +import com.google.common.base.Preconditions; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -26,26 +27,107 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.*; + +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.junit.Before; import org.junit.Test; +import java.util.Random; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +/** + * Need to cover the following combinations: + * 1. Block group size: + * 1.1 One byte + * 1.2 Smaller than cell + * 1.3 One full cell + * 1.4 x full cells, where x is smaller than number of data blocks + * 1.5 x full cells plus a partial cell + * 1.6 One full stripe + * 1.7 One full stripe plus a partial cell + * 1.8 One full stripe plus x full cells + * 1.9 One full stripe plus x full cells plus a partial cell + * 1.10 y full stripes, but smaller than full block group size + * 1.11 Full block group size + * + * 2. Byte range start + * 2.1 Zero + * 2.2 Within first cell + * 2.3 End of first cell + * 2.4 Start of a middle* cell in the first stripe (* neither first or last) + * 2.5 End of middle cell in the first stripe + * 2.6 Within a middle cell in the first stripe + * 2.7 Start of the last cell in the first stripe + * 2.8 Within the last cell in the first stripe + * 2.9 End of the last cell in the first stripe + * 2.10 Start of a middle stripe + * 2.11 Within a middle stripe + * 2.12 End of a middle stripe + * 2.13 Start of the last stripe + * 2.14 Within the last stripe + * 2.15 End of the last stripe (last byte) + * + * 3. Byte range length: same settings as block group size + * + * We should test in total 11 x 15 x 11 = 1815 combinations + * TODO: test parity block logic + */ public class TestStripedBlockUtil { private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; - private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; + private final short BLK_GROUP_WIDTH = DATA_BLK_NUM + PARITY_BLK_NUM; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE; + /** number of full stripes in a full block group */ + private final int BLK_GROUP_STRIPE_NUM = 16; + private final ECSchema SCEHMA = ErasureCodingSchemaManager. + getSystemDefaultSchema(); + private final Random random = new Random(); + + private int[] blockGroupSizes; + private int[] byteRangeStartOffsets; + private int[] byteRangeSizes; + + @Before + public void setup(){ + blockGroupSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE, + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE, + BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE}; + byteRangeStartOffsets = new int[] {0, getDelta(CELLSIZE), CELLSIZE - 1}; + byteRangeSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE, + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE, + BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE}; + } - private LocatedStripedBlock createDummyLocatedBlock() { + private int getDelta(int size) { + return 1 + random.nextInt(size - 2); + } + private byte hashIntToByte(int i) { + int BYTE_MASK = 0xff; + return (byte) (((i + 13) * 29) & BYTE_MASK); + } + + private LocatedStripedBlock createDummyLocatedBlock(int bgSize) { final long blockGroupID = -1048576; - DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_SIZE]; - String[] storageIDs = new String[BLK_GROUP_SIZE]; - StorageType[] storageTypes = new StorageType[BLK_GROUP_SIZE]; - int[] indices = new int[BLK_GROUP_SIZE]; - for (int i = 0; i < BLK_GROUP_SIZE; i++) { + DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_WIDTH]; + String[] storageIDs = new String[BLK_GROUP_WIDTH]; + StorageType[] storageTypes = new StorageType[BLK_GROUP_WIDTH]; + int[] indices = new int[BLK_GROUP_WIDTH]; + for (int i = 0; i < BLK_GROUP_WIDTH; i++) { indices[i] = (i + 2) % DATA_BLK_NUM; // Location port always equal to logical index of a block, // for easier verification @@ -53,13 +135,40 @@ private LocatedStripedBlock createDummyLocatedBlock() { storageIDs[i] = locs[i].getDatanodeUuid(); storageTypes[i] = StorageType.DISK; } - return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID), - locs, storageIDs, storageTypes, indices, 0, false, null); + return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID, + bgSize, 1001), locs, storageIDs, storageTypes, indices, 0, false, + null); + } + + private byte[][] createInternalBlkBuffers(int bgSize) { + byte[][] bufs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][]; + int[] pos = new int[DATA_BLK_NUM + PARITY_BLK_NUM]; + for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) { + int bufSize = (int) getInternalBlockLength( + bgSize, CELLSIZE, DATA_BLK_NUM, i); + bufs[i] = new byte[bufSize]; + pos[i] = 0; + } + int done = 0; + while (done < bgSize) { + Preconditions.checkState(done % CELLSIZE == 0); + StripingCell cell = new StripingCell(SCEHMA, done / CELLSIZE); + int idxInStripe = cell.idxInStripe; + int size = Math.min(CELLSIZE, bgSize - done); + for (int i = 0; i < size; i++) { + bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i); + } + done += size; + pos[idxInStripe] += size; + } + + return bufs; } @Test public void testParseDummyStripedBlock() { - LocatedStripedBlock lsb = createDummyLocatedBlock(); + LocatedStripedBlock lsb = createDummyLocatedBlock( + BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE); LocatedBlock[] blocks = parseStripedBlockGroup( lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length); @@ -68,14 +177,15 @@ public void testParseDummyStripedBlock() { assertEquals(i, BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock())); assertEquals(i * CELLSIZE, blocks[i].getStartOffset()); + /** TODO: properly define {@link LocatedBlock#offset} for internal blocks */ assertEquals(1, blocks[i].getLocations().length); assertEquals(i, blocks[i].getLocations()[0].getIpcPort()); assertEquals(i, blocks[i].getLocations()[0].getXferPort()); } } - private void verifyInternalBlocks (long numBytesInGroup, long[] expected) { - for (int i = 1; i < BLK_GROUP_SIZE; i++) { + private void verifyInternalBlocks (int numBytesInGroup, int[] expected) { + for (int i = 1; i < BLK_GROUP_WIDTH; i++) { assertEquals(expected[i], getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i)); } @@ -85,41 +195,85 @@ private void verifyInternalBlocks (long numBytesInGroup, long[] expected) { public void testGetInternalBlockLength () { // A small delta that is smaller than a cell final int delta = 10; - assert delta < CELLSIZE; // Block group is smaller than a cell verifyInternalBlocks(CELLSIZE - delta, - new long[] {CELLSIZE - delta, 0, 0, 0, 0, 0, + new int[] {CELLSIZE - delta, 0, 0, 0, 0, 0, CELLSIZE - delta, CELLSIZE - delta, CELLSIZE - delta}); // Block group is exactly as large as a cell verifyInternalBlocks(CELLSIZE, - new long[] {CELLSIZE, 0, 0, 0, 0, 0, + new int[] {CELLSIZE, 0, 0, 0, 0, 0, CELLSIZE, CELLSIZE, CELLSIZE}); // Block group is a little larger than a cell verifyInternalBlocks(CELLSIZE + delta, - new long[] {CELLSIZE, delta, 0, 0, 0, 0, + new int[] {CELLSIZE, delta, 0, 0, 0, 0, CELLSIZE, CELLSIZE, CELLSIZE}); // Block group contains multiple stripes and ends at stripe boundary verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE, - new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE}); // Block group contains multiple stripes and ends at cell boundary // (not ending at stripe boundary) verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE + CELLSIZE, - new long[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + new int[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE}); // Block group contains multiple stripes and doesn't end at cell boundary verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE - delta, - new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta, 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE}); } + /** + * Test dividing a byte range into aligned stripes and verify the aligned + * ranges can be translated back to the byte range. + */ + @Test + public void testDivideByteRangeIntoStripes() { + byte[] assembled = new byte[BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE]; + for (int bgSize : blockGroupSizes) { + LocatedStripedBlock blockGroup = createDummyLocatedBlock(bgSize); + byte[][] internalBlkBufs = createInternalBlkBuffers(bgSize); + for (int brStart : byteRangeStartOffsets) { + for (int brSize : byteRangeSizes) { + if (brStart + brSize > bgSize) { + continue; + } + AlignedStripe[] stripes = divideByteRangeIntoStripes(SCEHMA, + blockGroup, brStart, brStart + brSize - 1, assembled, 0); + + for (AlignedStripe stripe : stripes) { + for (int i = 0; i < DATA_BLK_NUM; i++) { + StripingChunk chunk = stripe.chunks[i]; + if (chunk == null || chunk.state != StripingChunk.REQUESTED) { + continue; + } + int done = 0; + for (int j = 0; j < chunk.getLengths().length; j++) { + System.arraycopy(internalBlkBufs[i], + (int) stripe.getOffsetInBlock() + done, assembled, + chunk.getOffsets()[j], chunk.getLengths()[j]); + done += chunk.getLengths()[j]; + } + } + } + for (int i = 0; i < brSize; i++) { + if (hashIntToByte(brStart + i) != assembled[i]) { + System.out.println("Oops"); + } + assertEquals("Byte at " + (brStart + i) + " should be the same", + hashIntToByte(brStart + i), assembled[i]); + } + } + } + } + } + } From 8c95673db47763650c4bb9f9bf39ef770aa73cdf Mon Sep 17 00:00:00 2001 From: boli2 Date: Tue, 19 May 2015 02:14:46 -0400 Subject: [PATCH 124/212] HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue configurable in DFSStripedOutputStream. Contributed by Li Bo --- .../hdfs/client/HdfsClientConfigKeys.java | 10 ++++++++ .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hadoop/hdfs/DFSStripedOutputStream.java | 19 +++++++++++---- .../hdfs/client/impl/DfsClientConf.java | 24 +++++++++++++++++++ 4 files changed, 51 insertions(+), 5 deletions(-) mode change 100644 => 100755 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 6006d71faf6a6..9373e98bf42c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -189,6 +189,16 @@ interface StripedRead { int THREADPOOL_SIZE_DEFAULT = 18; } + /** dfs.client.write.striped configuration properties */ + interface StripedWrite { + String PREFIX = Write.PREFIX + "striped."; + + String MAX_SECONDS_GET_STRIPED_BLOCK_KEY = PREFIX + "max-seconds-get-striped-block"; + int MAX_SECONDS_GET_STRIPED_BLOCK_DEFAULT = 90; + String MAX_SECONDS_GET_ENDED_BLOCK_KEY = PREFIX + "max-seconds-get-ended-block"; + int MAX_SECONDS_GET_ENDED_BLOCK_DEFAULT = 60; + } + /** dfs.http.client configuration properties */ interface HttpClient { String PREFIX = "dfs.http.client."; diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt old mode 100644 new mode 100755 index 3170e9bc0fbce..939ba89b7e745 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -225,3 +225,6 @@ (Yi Liu via jing9) HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz) + + HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue + configurable in DFSStripedOutputStream. (Li Bo) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index b99afab954703..a6480238b5909 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -33,6 +33,8 @@ import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -61,11 +63,14 @@ public class DFSStripedOutputStream extends DFSOutputStream { /** Coordinate the communication between the streamers. */ static class Coordinator { + private final DfsClientConf conf; private final List> endBlocks; private final List> stripedBlocks; private volatile boolean shouldLocateFollowingBlock = false; - Coordinator(final int numDataBlocks, final int numAllBlocks) { + Coordinator(final DfsClientConf conf, final int numDataBlocks, + final int numAllBlocks) { + this.conf = conf; endBlocks = new ArrayList<>(numDataBlocks); for (int i = 0; i < numDataBlocks; i++) { endBlocks.add(new LinkedBlockingQueue(1)); @@ -91,7 +96,9 @@ void putEndBlock(int i, ExtendedBlock block) { ExtendedBlock getEndBlock(int i) throws InterruptedIOException { try { - return endBlocks.get(i).poll(30, TimeUnit.SECONDS); + return endBlocks.get(i).poll( + conf.getStripedWriteMaxSecondsGetEndedBlock(), + TimeUnit.SECONDS); } catch (InterruptedException e) { throw DFSUtil.toInterruptedIOException( "getEndBlock interrupted, i=" + i, e); @@ -121,7 +128,9 @@ void putStripedBlock(int i, LocatedBlock block) throws IOException { LocatedBlock getStripedBlock(int i) throws IOException { final LocatedBlock lb; try { - lb = stripedBlocks.get(i).poll(90, TimeUnit.SECONDS); + lb = stripedBlocks.get(i).poll( + conf.getStripedWriteMaxSecondsGetStripedBlock(), + TimeUnit.SECONDS); } catch (InterruptedException e) { throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e); } @@ -133,7 +142,7 @@ LocatedBlock getStripedBlock(int i) throws IOException { } } - /** Buffers for writing the data and parity cells of a strip. */ + /** Buffers for writing the data and parity cells of a stripe. */ class CellBuffers { private final ByteBuffer[] buffers; private final byte[][] checksumArrays; @@ -228,7 +237,7 @@ private StripedDataStreamer getLeadingStreamer() { encoder = new RSRawEncoder(); encoder.initialize(numDataBlocks, numParityBlocks, cellSize); - coordinator = new Coordinator(numDataBlocks, numAllBlocks); + coordinator = new Coordinator(dfsClient.getConf(), numDataBlocks, numAllBlocks); try { cellBuffers = new CellBuffers(numParityBlocks); } catch (InterruptedException ie) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java index 32a3da0a7c28d..34ec06d999894 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java @@ -103,6 +103,9 @@ public class DfsClientConf { private final int hedgedReadThreadpoolSize; private final int stripedReadThreadpoolSize; + private final int stripedWriteMaxSecondsGetStripedBlock; + private final int stripedWriteMaxSecondsGetEndedBlock; + public DfsClientConf(Configuration conf) { // The hdfsTimeout is currently the same as the ipc timeout @@ -225,6 +228,13 @@ public DfsClientConf(Configuration conf) { Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY + " must be greater than 0."); + + stripedWriteMaxSecondsGetStripedBlock = conf.getInt( + HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_STRIPED_BLOCK_KEY, + HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_STRIPED_BLOCK_DEFAULT); + stripedWriteMaxSecondsGetEndedBlock = conf.getInt( + HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_ENDED_BLOCK_KEY, + HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_ENDED_BLOCK_DEFAULT); } private DataChecksum.Type getChecksumType(Configuration conf) { @@ -508,6 +518,20 @@ public int getStripedReadThreadpoolSize() { return stripedReadThreadpoolSize; } + /** + * @return stripedWriteMaxSecondsGetStripedBlock + */ + public int getStripedWriteMaxSecondsGetStripedBlock() { + return stripedWriteMaxSecondsGetStripedBlock; + } + + /** + * @return stripedWriteMaxSecondsGetEndedBlock + */ + public int getStripedWriteMaxSecondsGetEndedBlock() { + return stripedWriteMaxSecondsGetEndedBlock; + } + /** * @return the shortCircuitConf */ From 914580934c566cd18019035b244f82006868bd7b Mon Sep 17 00:00:00 2001 From: Walter Su Date: Tue, 19 May 2015 14:59:23 +0800 Subject: [PATCH 125/212] HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker. Contributed by Rakesh R. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 ++ .../hdfs/server/datanode/BPOfferService.java | 1 + .../erasurecode/ErasureCodingWorker.java | 59 +++++++++---------- 3 files changed, 35 insertions(+), 30 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 939ba89b7e745..1e7dbeaec11ad 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -218,6 +218,8 @@ HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng) + HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549) + HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. (Walter Su via jing9) @@ -228,3 +230,6 @@ HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue configurable in DFSStripedOutputStream. (Li Bo) + + HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker. + (Rakesh R via waltersu4549) 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 6606d0bbc7fde..d77b36dc1e02c 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 @@ -728,6 +728,7 @@ assert getBlockPoolId().equals(bp) : LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY"); Collection ecTasks = ((BlockECRecoveryCommand) cmd).getECTasks(); dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks); + break; default: LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index a1c0f722ce3a5..4723e9fca3bce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; @@ -88,12 +87,12 @@ * commands. */ public final class ErasureCodingWorker { - private final Log LOG = DataNode.LOG; + private static final Log LOG = DataNode.LOG; private final DataNode datanode; - private Configuration conf; + private final Configuration conf; - private ThreadPoolExecutor STRIPED_READ_TRHEAD_POOL; + private ThreadPoolExecutor STRIPED_READ_THREAD_POOL; private final int STRIPED_READ_THRESHOLD_MILLIS; private final int STRIPED_READ_BUFFER_SIZE; @@ -121,7 +120,10 @@ private RawErasureDecoder newDecoder() { } private void initializeStripedReadThreadPool(int num) { - STRIPED_READ_TRHEAD_POOL = new ThreadPoolExecutor(1, num, 60, + if (LOG.isDebugEnabled()) { + LOG.debug("Using striped reads; pool threads=" + num); + } + STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60, TimeUnit.SECONDS, new SynchronousQueue(), new Daemon.DaemonFactory() { private final AtomicInteger threadIndex = new AtomicInteger(0); @@ -141,7 +143,7 @@ public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) { super.rejectedExecution(runnable, e); } }); - STRIPED_READ_TRHEAD_POOL.allowCoreThreadTimeOut(true); + STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true); } /** @@ -231,23 +233,23 @@ private class ReconstructAndTransferBlock implements Runnable { // sources private final short[] liveIndices; - private DatanodeInfo[] sources; + private final DatanodeInfo[] sources; - private List stripedReaders; + private final List stripedReaders; // targets - private DatanodeInfo[] targets; - private StorageType[] targetStorageTypes; + private final DatanodeInfo[] targets; + private final StorageType[] targetStorageTypes; - private short[] targetIndices; - private ByteBuffer[] targetBuffers; + private final short[] targetIndices; + private final ByteBuffer[] targetBuffers; - private Socket[] targetSockets; - private DataOutputStream[] targetOutputStreams; - private DataInputStream[] targetInputStreams; + private final Socket[] targetSockets; + private final DataOutputStream[] targetOutputStreams; + private final DataInputStream[] targetInputStreams; - private long[] blockOffset4Targets; - private long[] seqNo4Targets; + private final long[] blockOffset4Targets; + private final long[] seqNo4Targets; private final int WRITE_PACKET_SIZE = 64 * 1024; private DataChecksum checksum; @@ -257,11 +259,11 @@ private class ReconstructAndTransferBlock implements Runnable { private int bytesPerChecksum; private int checksumSize; - private CachingStrategy cachingStrategy; + private final CachingStrategy cachingStrategy; - private Map, Integer> futures = new HashMap<>(); - private CompletionService readService = - new ExecutorCompletionService<>(STRIPED_READ_TRHEAD_POOL); + private final Map, Integer> futures = new HashMap<>(); + private final CompletionService readService = + new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL); ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) { ECSchema schema = recoveryInfo.getECSchema(); @@ -277,7 +279,8 @@ private class ReconstructAndTransferBlock implements Runnable { Preconditions.checkArgument(liveIndices.length >= dataBlkNum, "No enough live striped blocks."); - Preconditions.checkArgument(liveIndices.length == sources.length); + Preconditions.checkArgument(liveIndices.length == sources.length, + "liveBlockIndices and source dns should match"); targets = recoveryInfo.getTargetDnInfos(); targetStorageTypes = recoveryInfo.getTargetStorageTypes(); @@ -336,7 +339,6 @@ public void run() { if (nsuccess < dataBlkNum) { String error = "Can't find minimum sources required by " + "recovery, block id: " + blockGroup.getBlockId(); - LOG.warn(error); throw new IOException(error); } @@ -358,7 +360,6 @@ public void run() { boolean[] targetsStatus = new boolean[targets.length]; if (initTargetStreams(targetsStatus) == 0) { String error = "All targets are failed."; - LOG.warn(error); throw new IOException(error); } @@ -372,7 +373,6 @@ public void run() { if (nsuccess < dataBlkNum) { String error = "Can't read data from minimum number of sources " + "required by recovery, block id: " + blockGroup.getBlockId(); - LOG.warn(error); throw new IOException(error); } @@ -385,7 +385,6 @@ public void run() { // step3: transfer data if (transferData2Targets(targetsStatus) == 0) { String error = "Transfer failed for all targets."; - LOG.warn(error); throw new IOException(error); } @@ -906,11 +905,11 @@ private int initTargetStreams(boolean[] targetsStatus) { } private class StripedReader { - short index; - BlockReader blockReader; - ByteBuffer buffer; + private final short index; + private BlockReader blockReader; + private ByteBuffer buffer; - public StripedReader(short index) { + private StripedReader(short index) { this.index = index; } } From 91c81fdc24709b3caf1f6281c8879ffee08db956 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 19 May 2015 13:58:50 -0700 Subject: [PATCH 126/212] HDFS-8375. Add cellSize as an XAttr to ECZone. Contributed by Vinayakumar B. --- .../hadoop/hdfs/protocol/HdfsFileStatus.java | 14 ++++-- .../SnapshottableDirectoryStatus.java | 2 +- .../hadoop/hdfs/web/JsonUtilClient.java | 2 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 8 +-- .../org/apache/hadoop/hdfs/DFSClient.java | 7 +-- .../hadoop/hdfs/DFSStripedInputStream.java | 10 ++-- .../hadoop/hdfs/DFSStripedOutputStream.java | 2 +- .../hadoop/hdfs/DistributedFileSystem.java | 9 ++-- .../hadoop/hdfs/protocol/ClientProtocol.java | 2 +- .../hdfs/protocol/ErasureCodingZoneInfo.java | 14 +++++- .../hdfs/protocol/HdfsLocatedFileStatus.java | 5 +- ...amenodeProtocolServerSideTranslatorPB.java | 3 +- .../ClientNamenodeProtocolTranslatorPB.java | 5 +- .../hadoop/hdfs/protocolPB/PBHelper.java | 15 ++++-- .../server/blockmanagement/BlockManager.java | 12 +++-- .../blockmanagement/DatanodeDescriptor.java | 4 +- .../erasurecode/ErasureCodingWorker.java | 2 +- .../namenode/ErasureCodingZoneManager.java | 40 ++++++++++++--- .../namenode/FSDirStatAndListingOp.java | 21 +++++--- .../hdfs/server/namenode/FSDirectory.java | 4 +- .../hdfs/server/namenode/FSNamesystem.java | 49 +++++++------------ .../server/namenode/NameNodeRpcServer.java | 5 +- .../hdfs/server/namenode/Namesystem.java | 8 +-- .../protocol/BlockECRecoveryCommand.java | 26 +++++----- .../hdfs/tools/erasurecode/ECCommand.java | 14 +++++- .../hadoop/hdfs/util/StripedBlockUtil.java | 49 +++++++++---------- .../src/main/proto/erasurecoding.proto | 3 ++ .../hadoop-hdfs/src/main/proto/hdfs.proto | 3 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 2 +- .../hadoop/hdfs/TestDFSClientRetries.java | 4 +- .../hdfs/TestDFSStripedInputStream.java | 10 ++-- .../hdfs/TestDFSStripedOutputStream.java | 2 +- ...TestDFSStripedOutputStreamWithFailure.java | 2 +- .../hadoop/hdfs/TestEncryptionZones.java | 2 +- .../hadoop/hdfs/TestErasureCodingZones.java | 18 +++---- .../hdfs/TestFileStatusWithECschema.java | 2 +- .../org/apache/hadoop/hdfs/TestLease.java | 4 +- .../hadoop/hdfs/TestRecoverStripedFile.java | 2 +- .../hadoop/hdfs/TestWriteReadStripedFile.java | 2 +- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 6 ++- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 4 +- .../hdfs/server/namenode/TestFSImage.java | 4 +- .../hadoop/hdfs/server/namenode/TestFsck.java | 4 +- .../namenode/TestQuotaWithStripedBlocks.java | 2 +- ...stOfflineImageViewerWithStripedBlocks.java | 2 +- .../hdfs/util/TestStripedBlockUtil.java | 4 +- .../apache/hadoop/hdfs/web/TestJsonUtil.java | 2 +- 48 files changed, 244 insertions(+), 174 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java index f07973a8e20ea..8c902b4aae1d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java @@ -49,7 +49,8 @@ public class HdfsFileStatus { private final FileEncryptionInfo feInfo; - private final ECSchema schema; + private final ECSchema ecSchema; + private final int stripeCellSize; // Used by dir, not including dot and dotdot. Always zero for a regular file. private final int childrenNum; @@ -76,7 +77,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo, - byte storagePolicy, ECSchema schema) { + byte storagePolicy, ECSchema ecSchema, int stripeCellSize) { this.length = length; this.isdir = isdir; this.block_replication = (short)block_replication; @@ -96,7 +97,8 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, this.childrenNum = childrenNum; this.feInfo = feInfo; this.storagePolicy = storagePolicy; - this.schema = schema; + this.ecSchema = ecSchema; + this.stripeCellSize = stripeCellSize; } /** @@ -255,7 +257,11 @@ public final FileEncryptionInfo getFileEncryptionInfo() { } public ECSchema getECSchema() { - return schema; + return ecSchema; + } + + public int getStripeCellSize() { + return stripeCellSize; } public final int getChildrenNum() { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java index 813ea266cd07f..a6c7b10c91546 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java @@ -61,7 +61,7 @@ public SnapshottableDirectoryStatus(long modification_time, long access_time, int snapshotNumber, int snapshotQuota, byte[] parentFullPath) { this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time, access_time, permission, owner, group, null, localName, inodeId, - childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); + childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0); this.snapshotNumber = snapshotNumber; this.snapshotQuota = snapshotQuota; this.parentFullPath = parentFullPath; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java index 62f679b727a87..5635c1b7d803a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java @@ -132,7 +132,7 @@ static HdfsFileStatus toFileStatus(final Map json, boolean includesType) { blockSize, mTime, aTime, permission, owner, group, symlink, DFSUtilClient.string2Bytes(localName), fileId, childrenNum, null, - storagePolicy, null); + storagePolicy, null, 0); } /** Convert a Json map to an ExtendedBlock object. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 1e7dbeaec11ad..d6c9dba6958e7 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -227,9 +227,11 @@ (Yi Liu via jing9) HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz) - - HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue - configurable in DFSStripedOutputStream. (Li Bo) + + HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue + configurable in DFSStripedOutputStream. (Li Bo) HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker. (Rakesh R via waltersu4549) + + HDFS-8375. Add cellSize as an XAttr to ECZone. ( Vinayakumar B via zhz). diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index ffeb568cff837..42f48f0959666 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1197,7 +1197,8 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) if (fileInfo != null) { ECSchema schema = fileInfo.getECSchema(); if (schema != null) { - return new DFSStripedInputStream(this, src, verifyChecksum, schema); + return new DFSStripedInputStream(this, src, verifyChecksum, schema, + fileInfo.getStripeCellSize()); } } return new DFSInputStream(this, src, verifyChecksum); @@ -3009,12 +3010,12 @@ public RemoteIterator listEncryptionZones() return new EncryptionZoneIterator(namenode, traceSampler); } - public void createErasureCodingZone(String src, ECSchema schema) + public void createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException { checkOpen(); TraceScope scope = getPathTraceScope("createErasureCodingZone", src); try { - namenode.createErasureCodingZone(src, schema); + namenode.createErasureCodingZone(src, schema, cellSize); } catch (RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, SafeModeException.class, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 744d5863afeaf..3b7eb5881402b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -125,12 +125,12 @@ boolean include(long pos) { private final CompletionService readingService; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, - ECSchema schema) throws IOException { + ECSchema schema, int cellSize) throws IOException { super(dfsClient, src, verifyChecksum); assert schema != null; this.schema = schema; - cellSize = schema.getChunkSize(); + this.cellSize = cellSize; dataBlkNum = (short) schema.getNumDataUnits(); parityBlkNum = (short) schema.getNumParityUnits(); groupSize = dataBlkNum; @@ -189,7 +189,7 @@ private synchronized void blockSeekTo(long target) throws IOException { targetBlockGroup, cellSize, dataBlkNum, parityBlkNum); // The purpose is to get start offset into each block. long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema, - targetBlockGroup, offsetIntoBlockGroup); + cellSize, targetBlockGroup, offsetIntoBlockGroup); Preconditions.checkNotNull(offsetsForInternalBlocks); final ReaderRetryPolicy retry = new ReaderRetryPolicy(); @@ -514,8 +514,8 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, // Refresh the striped block group LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset); - AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, blockGroup, - start, end, buf, offset); + AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, cellSize, + blockGroup, start, end, buf, offset); for (AlignedStripe stripe : stripes) { fetchOneStripe(blockGroup, buf, stripe, corruptedBlockMap); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index a6480238b5909..4399a37f85931 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -230,7 +230,7 @@ private StripedDataStreamer getLeadingStreamer() { final ECSchema schema = stat.getECSchema(); final int numParityBlocks = schema.getNumParityUnits(); - cellSize = schema.getChunkSize(); + cellSize = stat.getStripeCellSize(); numDataBlocks = schema.getNumDataUnits(); numAllBlocks = numDataBlocks + numParityBlocks; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 0cd8334cf2f5d..2e21372593aa9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -2281,16 +2281,17 @@ public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid) * * @param path Directory to create the ec zone * @param schema ECSchema for the zone. If not specified default will be used. + * @param cellSize Cellsize for the striped erasure coding * @throws IOException */ - public void createErasureCodingZone(final Path path, final ECSchema schema) - throws IOException { + public void createErasureCodingZone(final Path path, final ECSchema schema, + final int cellSize) throws IOException { Path absF = fixRelativePart(path); new FileSystemLinkResolver() { @Override public Void doCall(final Path p) throws IOException, UnresolvedLinkException { - dfs.createErasureCodingZone(getPathName(p), schema); + dfs.createErasureCodingZone(getPathName(p), schema, cellSize); return null; } @@ -2298,7 +2299,7 @@ public Void doCall(final Path p) throws IOException, public Void next(final FileSystem fs, final Path p) throws IOException { if (fs instanceof DistributedFileSystem) { DistributedFileSystem myDfs = (DistributedFileSystem) fs; - myDfs.createErasureCodingZone(p, schema); + myDfs.createErasureCodingZone(p, schema, cellSize); return null; } throw new UnsupportedOperationException( 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 b0b457cf415cc..4f985ba60736a 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 @@ -1463,7 +1463,7 @@ public List listXAttrs(String src) * default */ @AtMostOnce - public void createErasureCodingZone(String src, ECSchema schema) + public void createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java index ec0efbd345466..282eeaf60bd88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java @@ -25,10 +25,12 @@ public class ErasureCodingZoneInfo { private String dir; private ECSchema schema; + private int cellSize; - public ErasureCodingZoneInfo(String dir, ECSchema schema) { + public ErasureCodingZoneInfo(String dir, ECSchema schema, int cellSize) { this.dir = dir; this.schema = schema; + this.cellSize = cellSize; } /** @@ -49,8 +51,16 @@ public ECSchema getSchema() { return schema; } + /** + * Get cellSize for the EC Zone + */ + public int getCellSize() { + return cellSize; + } + @Override public String toString() { - return "Dir: " + getDir() + ", Schema: " + schema; + return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: " + + cellSize; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java index 9194d26258d74..4701538de9e7e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java @@ -59,10 +59,11 @@ public HdfsLocatedFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, LocatedBlocks locations, - int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy, ECSchema schema) { + int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy, + ECSchema schema, int stripeCellSize) { super(length, isdir, block_replication, blocksize, modification_time, access_time, permission, owner, group, symlink, path, fileId, - childrenNum, feInfo, storagePolicy, schema); + childrenNum, feInfo, storagePolicy, schema, stripeCellSize); this.locations = locations; } 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 bae753b72bb88..863b217254dac 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 @@ -1408,7 +1408,8 @@ public CreateErasureCodingZoneResponseProto createErasureCodingZone( try { ECSchema schema = req.hasSchema() ? PBHelper.convertECSchema(req .getSchema()) : null; - server.createErasureCodingZone(req.getSrc(), schema); + int cellSize = req.hasCellSize() ? req.getCellSize() : 0; + server.createErasureCodingZone(req.getSrc(), schema, cellSize); return CreateErasureCodingZoneResponseProto.newBuilder().build(); } 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 67b1457414cd0..336e3a25a8ea4 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 @@ -1422,7 +1422,7 @@ public BatchedEntries listEncryptionZones(long id) } @Override - public void createErasureCodingZone(String src, ECSchema schema) + public void createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException { final CreateErasureCodingZoneRequestProto.Builder builder = CreateErasureCodingZoneRequestProto.newBuilder(); @@ -1430,6 +1430,9 @@ public void createErasureCodingZone(String src, ECSchema schema) if (schema != null) { builder.setSchema(PBHelper.convertECSchema(schema)); } + if (cellSize > 0) { + builder.setCellSize(cellSize); + } CreateErasureCodingZoneRequestProto req = builder.build(); try { rpcProxy.createErasureCodingZone(null, req); 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 a6a356cc9ce8d..4d0f8710cf239 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 @@ -1506,7 +1506,8 @@ public static HdfsFileStatus convert(HdfsFileStatusProto fs) { fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null, fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy() : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, - fs.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null); + fs.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null, + fs.hasStripeCellSize() ? fs.getStripeCellSize() : 0); } public static SnapshottableDirectoryStatus convert( @@ -1570,6 +1571,7 @@ public static HdfsFileStatusProto convert(HdfsFileStatus fs) { if(fs.getECSchema() != null) { builder.setEcSchema(PBHelper.convertECSchema(fs.getECSchema())); } + builder.setStripeCellSize(fs.getStripeCellSize()); return builder.build(); } @@ -3157,12 +3159,14 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) { return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir()) - .setSchema(convertECSchema(ecZoneInfo.getSchema())).build(); + .setSchema(convertECSchema(ecZoneInfo.getSchema())) + .setCellSize(ecZoneInfo.getCellSize()).build(); } public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) { return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(), - convertECSchema(ecZoneInfoProto.getSchema())); + convertECSchema(ecZoneInfoProto.getSchema()), + ecZoneInfoProto.getCellSize()); } public static BlockECRecoveryInfo convertBlockECRecoveryInfo( @@ -3196,9 +3200,11 @@ public static BlockECRecoveryInfo convertBlockECRecoveryInfo( } ECSchema ecSchema = convertECSchema(blockEcRecoveryInfoProto.getEcSchema()); + int cellSize = blockEcRecoveryInfoProto.getCellSize(); return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos, - targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema); + targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema, + cellSize); } public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( @@ -3224,6 +3230,7 @@ public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices)); builder.setEcSchema(convertECSchema(blockEcRecoveryInfo.getECSchema())); + builder.setCellSize(blockEcRecoveryInfo.getCellSize()); return builder.build(); } 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 d296aa8707327..28e7b89e06910 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 @@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -1560,14 +1561,14 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { assert rw instanceof ErasureCodingWork; assert rw.targets.length > 0; String src = block.getBlockCollection().getName(); - ECSchema ecSchema = null; + ErasureCodingZoneInfo ecZoneInfo = null; try { - ecSchema = namesystem.getECSchemaForPath(src); + ecZoneInfo = namesystem.getErasureCodingZoneInfoForPath(src); } catch (IOException e) { blockLog - .warn("Failed to get the EC schema for the file {} ", src); + .warn("Failed to get the EC zone info for the file {} ", src); } - if (ecSchema == null) { + if (ecZoneInfo == null) { blockLog.warn("No EC schema found for the file {}. " + "So cannot proceed for recovery", src); // TODO: we may have to revisit later for what we can do better to @@ -1577,7 +1578,8 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded( new ExtendedBlock(namesystem.getBlockPoolId(), block), rw.srcNodes, rw.targets, - ((ErasureCodingWork) rw).liveBlockIndicies, ecSchema); + ((ErasureCodingWork) rw).liveBlockIndicies, + ecZoneInfo.getSchema(), ecZoneInfo.getCellSize()); } else { rw.srcNodes[0].addBlockToBeReplicated(block, targets); } 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 83d33035bdd6f..47bc7652e0981 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 @@ -610,10 +610,10 @@ void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { */ void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets, - short[] liveBlockIndices, ECSchema ecSchema) { + short[] liveBlockIndices, ECSchema ecSchema, int cellSize) { assert (block != null && sources != null && sources.length > 0); BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets, - liveBlockIndices, ecSchema); + liveBlockIndices, ecSchema, cellSize); erasurecodeBlocks.offer(task); BlockManager.LOG.debug("Adding block recovery task " + task + "to " + getName() + ", current queue size is " + erasurecodeBlocks.size()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 4723e9fca3bce..d227de85cf2e0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -269,7 +269,7 @@ private class ReconstructAndTransferBlock implements Runnable { ECSchema schema = recoveryInfo.getECSchema(); dataBlkNum = schema.getNumDataUnits(); parityBlkNum = schema.getNumParityUnits(); - cellSize = schema.getChunkSize(); + cellSize = recoveryInfo.getCellSize(); blockGroup = recoveryInfo.getExtendedBlock(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 6b6add7497a13..371b8acf55f92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -19,12 +19,20 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; + import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.erasurecode.ECSchema; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.util.ArrayList; import java.util.EnumSet; @@ -78,17 +86,21 @@ ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { : inode.getXAttrFeature().getXAttrs(); for (XAttr xAttr : xAttrs) { if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { - String schemaName = new String(xAttr.getValue()); + ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue()); + DataInputStream dIn=new DataInputStream(bIn); + int cellSize = WritableUtils.readVInt(dIn); + String schemaName = WritableUtils.readString(dIn); ECSchema schema = dir.getFSNamesystem().getECSchemaManager() .getSchema(schemaName); - return new ErasureCodingZoneInfo(inode.getFullPathName(), schema); + return new ErasureCodingZoneInfo(inode.getFullPathName(), schema, + cellSize); } } } return null; } - XAttr createErasureCodingZone(String src, ECSchema schema) + XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException { assert dir.hasWriteLock(); final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false); @@ -113,10 +125,24 @@ XAttr createErasureCodingZone(String src, ECSchema schema) schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); } - // Now persist the schema name in xattr - byte[] schemaBytes = schema.getSchemaName().getBytes(); - final XAttr ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE, - schemaBytes); + if (cellSize <= 0) { + cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + } + + // Write the cellsize first and then schema name + final XAttr ecXAttr; + DataOutputStream dOut = null; + try { + ByteArrayOutputStream bOut = new ByteArrayOutputStream(); + dOut = new DataOutputStream(bOut); + WritableUtils.writeVInt(dOut, cellSize); + // Now persist the schema name in xattr + WritableUtils.writeString(dOut, schema.getSchemaName()); + ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE, + bOut.toByteArray()); + } finally { + IOUtils.closeStream(dOut); + } final List xattrs = Lists.newArrayListWithCapacity(1); xattrs.add(ecXAttr); FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index 7133cf1950be4..eba5013a5a5e5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -317,7 +318,7 @@ static HdfsFileStatus getFileInfo( if (fsd.getINode4DotSnapshot(srcs) != null) { return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null, HdfsFileStatus.EMPTY_NAME, -1L, 0, null, - HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); + HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0); } return null; } @@ -385,8 +386,10 @@ static HdfsFileStatus createFileStatus( final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); - final ECSchema schema = fsd.getECSchema(iip); - + final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip); + final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null; + final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0; + if (node.isFile()) { final INodeFile fileNode = node.asFile(); size = fileNode.computeFileSize(snapshot); @@ -417,7 +420,8 @@ static HdfsFileStatus createFileStatus( childrenNum, feInfo, storagePolicy, - schema); + schema, + cellSize); } private static INodeAttributes getINodeAttributes( @@ -464,8 +468,10 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - final ECSchema schema = fsd.getECSchema(iip); - + final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip); + final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null; + final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0; + HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(size, node.isDirectory(), replication, blocksize, node.getModificationTime(snapshot), @@ -473,7 +479,8 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( getPermissionForFileStatus(nodeAttrs, isEncrypted), nodeAttrs.getUserName(), nodeAttrs.getGroupName(), node.isSymlink() ? node.asSymlink().getSymlink() : null, path, - node.getId(), loc, childrenNum, feInfo, storagePolicy, schema); + node.getId(), loc, childrenNum, feInfo, storagePolicy, schema, + cellSize); // Set caching information for the located blocks. if (loc != null) { CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager(); 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 bb8282d1da01a..735ea5a928ed6 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 @@ -1230,11 +1230,11 @@ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId, } } - XAttr createErasureCodingZone(String src, ECSchema schema) + XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException { writeLock(); try { - return ecZoneManager.createErasureCodingZone(src, schema); + return ecZoneManager.createErasureCodingZone(src, schema, cellSize); } finally { writeUnlock(); } 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 6643dfe813dec..60f86d6fa3274 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 @@ -7555,14 +7555,14 @@ BatchedListEntries listEncryptionZones(long prevId) * @param srcArg the path of a directory which will be the root of the * erasure coding zone. The directory must be empty. * @param schema ECSchema for the erasure coding zone - * + * @param cellSize Cell size of stripe * @throws AccessControlException if the caller is not the superuser. * @throws UnresolvedLinkException if the path can't be resolved. * @throws SafeModeException if the Namenode is in safe mode. */ void createErasureCodingZone(final String srcArg, final ECSchema schema, - final boolean logRetryCache) throws IOException, UnresolvedLinkException, - SafeModeException, AccessControlException { + int cellSize, final boolean logRetryCache) throws IOException, + UnresolvedLinkException, SafeModeException, AccessControlException { String src = srcArg; HdfsFileStatus resultingStat = null; FSPermissionChecker pc = null; @@ -7585,7 +7585,7 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, checkNameNodeSafeMode("Cannot create erasure coding zone on " + src); src = dir.resolvePath(pc, src, pathComponents); - final XAttr ecXAttr = dir.createErasureCodingZone(src, schema); + final XAttr ecXAttr = dir.createErasureCodingZone(src, schema, cellSize); List xAttrs = Lists.newArrayListWithCapacity(1); xAttrs.add(ecXAttr); getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); @@ -7604,9 +7604,9 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, */ ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException, UnresolvedLinkException, IOException { - ECSchema schema = getECSchemaForPath(src); - if (schema != null) { - return new ErasureCodingInfo(src, schema); + ErasureCodingZoneInfo zoneInfo = getErasureCodingZoneInfo(src); + if (zoneInfo != null) { + return new ErasureCodingInfo(src, zoneInfo.getSchema()); } return null; } @@ -7614,21 +7614,13 @@ ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException /** * Get the erasure coding zone information for specified path */ - ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException, - UnresolvedLinkException, IOException { + ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) + throws AccessControlException, UnresolvedLinkException, IOException { checkOperation(OperationCategory.READ); - final byte[][] pathComponents = FSDirectory - .getPathComponentsForReservedPath(src); - final FSPermissionChecker pc = getPermissionChecker(); readLock(); try { checkOperation(OperationCategory.READ); - src = dir.resolvePath(pc, src, pathComponents); - final INodesInPath iip = dir.getINodesInPath(src, true); - if (isPermissionEnabled) { - dir.checkPathAccess(pc, iip, FsAction.READ); - } - return dir.getECZoneInfo(iip); + return getErasureCodingZoneInfoForPath(src); } finally { readUnlock(); } @@ -7849,24 +7841,17 @@ private static void enableAsyncAuditLog() { } @Override - public ECSchema getECSchemaForPath(String src) throws IOException { - checkOperation(OperationCategory.READ); + public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src) + throws IOException { final byte[][] pathComponents = FSDirectory .getPathComponentsForReservedPath(src); final FSPermissionChecker pc = getPermissionChecker(); - readLock(); - try { - checkOperation(OperationCategory.READ); - src = dir.resolvePath(pc, src, pathComponents); - final INodesInPath iip = dir.getINodesInPath(src, true); - if (isPermissionEnabled) { - dir.checkPathAccess(pc, iip, FsAction.READ); - } - // Get schema set for the zone - return dir.getECSchema(iip); - } finally { - readUnlock(); + src = dir.resolvePath(pc, src, pathComponents); + final INodesInPath iip = dir.getINodesInPath(src, true); + if (isPermissionEnabled) { + dir.checkPathAccess(pc, iip, FsAction.READ); } + return dir.getECZoneInfo(iip); } } 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 5a69b2f2e0279..747f5283d7710 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 @@ -1824,7 +1824,7 @@ public BatchedEntries listEncryptionZones( } @Override // ClientProtocol - public void createErasureCodingZone(String src, ECSchema schema) + public void createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException { checkNNStartup(); final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); @@ -1833,7 +1833,8 @@ public void createErasureCodingZone(String src, ECSchema schema) } boolean success = false; try { - namesystem.createErasureCodingZone(src, schema, cacheEntry != null); + namesystem.createErasureCodingZone(src, schema, cellSize, + cacheEntry != null); success = true; } finally { RetryCache.setState(cacheEntry, success); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java index e6c7fc0f17a34..a32e800e5129f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.util.RwLock; @@ -51,12 +52,13 @@ public interface Namesystem extends RwLock, SafeMode { public boolean isInSnapshot(BlockCollection bc); /** - * Gets the ECSchema for the specified path + * Gets the ECZone info for path * * @param src * - path - * @return ECSchema + * @return {@link ErasureCodingZoneInfo} * @throws IOException */ - public ECSchema getECSchemaForPath(String src) throws IOException; + public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src) + throws IOException; } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java index 61e49e933de66..56a1546d29966 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java @@ -78,25 +78,22 @@ public static class BlockECRecoveryInfo { private StorageType[] targetStorageTypes; private final short[] liveBlockIndices; private final ECSchema ecSchema; + private final int cellSize; public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices, - ECSchema ecSchema) { - this.block = block; - this.sources = sources; - this.targets = DatanodeStorageInfo.toDatanodeInfos(targetDnStorageInfo); - this.targetStorageIDs = DatanodeStorageInfo - .toStorageIDs(targetDnStorageInfo); - this.targetStorageTypes = DatanodeStorageInfo - .toStorageTypes(targetDnStorageInfo); - this.liveBlockIndices = liveBlockIndices; - this.ecSchema = ecSchema; + ECSchema ecSchema, int cellSize) { + this(block, sources, DatanodeStorageInfo + .toDatanodeInfos(targetDnStorageInfo), DatanodeStorageInfo + .toStorageIDs(targetDnStorageInfo), DatanodeStorageInfo + .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecSchema, + cellSize); } - + public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, DatanodeInfo[] targets, String[] targetStorageIDs, StorageType[] targetStorageTypes, short[] liveBlockIndices, - ECSchema ecSchema) { + ECSchema ecSchema, int cellSize) { this.block = block; this.sources = sources; this.targets = targets; @@ -104,6 +101,7 @@ public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, this.targetStorageTypes = targetStorageTypes; this.liveBlockIndices = liveBlockIndices; this.ecSchema = ecSchema; + this.cellSize = cellSize; } public ExtendedBlock getExtendedBlock() { @@ -134,6 +132,10 @@ public ECSchema getECSchema() { return ecSchema; } + public int getCellSize() { + return cellSize; + } + @Override public String toString() { return new StringBuilder().append("BlockECRecoveryInfo(\n ") diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index 2b6a6a52062e4..2d8220851f118 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.shell.PathData; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.StringUtils; @@ -88,14 +89,23 @@ static class CreateECZoneCommand extends ECCommand { + "Options :\n" + " -s : EC schema name to encode files. " + "If not passed default schema will be used\n" + + " -c : cell size to use for striped encoding files." + + " If not passed default cellsize of " + + HdfsConstants.BLOCK_STRIPED_CELL_SIZE + " will be used\n" + " : Path to an empty directory. Under this directory " + "files will be encoded using specified schema"; private String schemaName; + private int cellSize = 0; private ECSchema schema = null; @Override protected void processOptions(LinkedList args) throws IOException { schemaName = StringUtils.popOptionWithArgument("-s", args); + String cellSizeStr = StringUtils.popOptionWithArgument("-c", args); + if (cellSizeStr != null) { + cellSize = (int) StringUtils.TraditionalBinaryPrefix + .string2long(cellSizeStr); + } if (args.isEmpty()) { throw new HadoopIllegalArgumentException(" is missing"); } @@ -131,7 +141,7 @@ protected void processPath(PathData item) throws IOException { throw new HadoopIllegalArgumentException(sb.toString()); } } - dfs.createErasureCodingZone(item.path, schema); + dfs.createErasureCodingZone(item.path, schema, cellSize); out.println("EC Zone created successfully at " + item.path); } catch (IOException e) { throw new IOException("Unable to create EC zone for the path " @@ -213,4 +223,4 @@ protected void processOptions(LinkedList args) throws IOException { out.println(sb.toString()); } } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 2fa3fdf6ae8c8..6f7dcb1478685 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -306,6 +306,7 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf, * {@link AlignedStripe}. * @param ecSchema The codec schema for the file, which carries the numbers * of data / parity blocks, as well as cell size + * @param cellSize Cell size of stripe * @param blockGroup The striped block group * @param rangeStartInBlockGroup The byte range's start offset in block group * @param rangeEndInBlockGroup The byte range's end offset in block group @@ -315,28 +316,29 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf, * At most 5 stripes will be generated from each logical range, as * demonstrated in the header of {@link AlignedStripe}. */ - public static AlignedStripe[] divideByteRangeIntoStripes ( - ECSchema ecSchema, LocatedStripedBlock blockGroup, + public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, + int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf, int offsetInBuf) { // TODO: change ECSchema naming to use cell size instead of chunk size // Step 0: analyze range and calculate basic parameters - int cellSize = ecSchema.getChunkSize(); int dataBlkNum = ecSchema.getNumDataUnits(); // Step 1: map the byte range to StripingCells - StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, blockGroup, - rangeStartInBlockGroup, rangeEndInBlockGroup); + StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize, + blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup); // Step 2: get the unmerged ranges on each internal block - VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cells); + VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize, + cells); // Step 3: merge into at most 5 stripes AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); // Step 4: calculate each chunk's position in destination buffer - calcualteChunkPositionsInBuf(ecSchema, stripes, cells, buf, offsetInBuf); + calcualteChunkPositionsInBuf(ecSchema, cellSize, stripes, cells, buf, + offsetInBuf); // Step 5: prepare ALLZERO blocks prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum); @@ -351,19 +353,18 @@ public static AlignedStripe[] divideByteRangeIntoStripes ( */ @VisibleForTesting private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, - LocatedStripedBlock blockGroup, + int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, long rangeEndInBlockGroup) { Preconditions.checkArgument( rangeStartInBlockGroup <= rangeEndInBlockGroup && rangeEndInBlockGroup < blockGroup.getBlockSize()); - int cellSize = ecSchema.getChunkSize(); int len = (int) (rangeEndInBlockGroup - rangeStartInBlockGroup + 1); int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize); int numCells = lastCellIdxInBG - firstCellIdxInBG + 1; StripingCell[] cells = new StripingCell[numCells]; - cells[0] = new StripingCell(ecSchema, firstCellIdxInBG); - cells[numCells - 1] = new StripingCell(ecSchema, lastCellIdxInBG); + cells[0] = new StripingCell(ecSchema, cellSize, firstCellIdxInBG); + cells[numCells - 1] = new StripingCell(ecSchema, cellSize, lastCellIdxInBG); cells[0].offset = (int) (rangeStartInBlockGroup % cellSize); cells[0].size = @@ -373,7 +374,7 @@ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, } for (int i = 1; i < numCells - 1; i++) { - cells[i] = new StripingCell(ecSchema, i + firstCellIdxInBG); + cells[i] = new StripingCell(ecSchema, cellSize, i + firstCellIdxInBG); } return cells; @@ -383,18 +384,16 @@ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, * Given a logical start offset in a block group, calculate the physical * start offset into each stored internal block. */ - public static long[] getStartOffsetsForInternalBlocks( - ECSchema ecSchema, LocatedStripedBlock blockGroup, - long rangeStartInBlockGroup) { + public static long[] getStartOffsetsForInternalBlocks(ECSchema ecSchema, + int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup) { Preconditions.checkArgument( rangeStartInBlockGroup < blockGroup.getBlockSize()); int dataBlkNum = ecSchema.getNumDataUnits(); int parityBlkNum = ecSchema.getNumParityUnits(); - int cellSize = ecSchema.getChunkSize(); long[] startOffsets = new long[dataBlkNum + parityBlkNum]; Arrays.fill(startOffsets, -1L); int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); - StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG); + StripingCell firstCell = new StripingCell(ecSchema, cellSize, firstCellIdxInBG); firstCell.offset = (int) (rangeStartInBlockGroup % cellSize); startOffsets[firstCell.idxInStripe] = firstCell.idxInInternalBlk * cellSize + firstCell.offset; @@ -404,7 +403,7 @@ public static long[] getStartOffsetsForInternalBlocks( if (idx * cellSize >= blockGroup.getBlockSize()) { break; } - StripingCell cell = new StripingCell(ecSchema, idx); + StripingCell cell = new StripingCell(ecSchema, cellSize, idx); startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize; if (startOffsets[cell.idxInStripe] < earliestStart) { earliestStart = startOffsets[cell.idxInStripe]; @@ -422,8 +421,7 @@ public static long[] getStartOffsetsForInternalBlocks( */ @VisibleForTesting private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema, - StripingCell[] cells) { - int cellSize = ecSchema.getChunkSize(); + int cellSize, StripingCell[] cells) { int dataBlkNum = ecSchema.getNumDataUnits(); int parityBlkNum = ecSchema.getNumParityUnits(); @@ -486,7 +484,7 @@ private static AlignedStripe[] mergeRangesForInternalBlocks( } private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, - AlignedStripe[] stripes, StripingCell[] cells, byte[] buf, + int cellSize, AlignedStripe[] stripes, StripingCell[] cells, byte[] buf, int offsetInBuf) { /** * | <--------------- AlignedStripe --------------->| @@ -505,7 +503,6 @@ private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, * * Cell indexing convention defined in {@link StripingCell} */ - int cellSize = ecSchema.getChunkSize(); int done = 0; for (StripingCell cell : cells) { long cellStart = cell.idxInInternalBlk * cellSize + cell.offset; @@ -587,17 +584,17 @@ static class StripingCell { int offset; int size; - StripingCell(ECSchema ecSchema, int idxInBlkGroup) { + StripingCell(ECSchema ecSchema, int cellSize, int idxInBlkGroup) { this.schema = ecSchema; this.idxInBlkGroup = idxInBlkGroup; this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits(); this.idxInStripe = idxInBlkGroup - this.idxInInternalBlk * ecSchema.getNumDataUnits(); this.offset = 0; - this.size = ecSchema.getChunkSize(); + this.size = cellSize; } - StripingCell(ECSchema ecSchema, int idxInInternalBlk, + StripingCell(ECSchema ecSchema, int cellSize, int idxInInternalBlk, int idxInStripe) { this.schema = ecSchema; this.idxInInternalBlk = idxInInternalBlk; @@ -605,7 +602,7 @@ static class StripingCell { this.idxInBlkGroup = idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe; this.offset = 0; - this.size = ecSchema.getChunkSize(); + this.size = cellSize; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 2302d1d4c4c5b..058ed96ee3a20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -37,11 +37,13 @@ message ErasureCodingInfoProto { message ErasureCodingZoneInfoProto { required string dir = 1; required ECSchemaProto schema = 2; + required uint32 cellSize = 3; } message CreateErasureCodingZoneRequestProto { required string src = 1; optional ECSchemaProto schema = 2; + optional uint32 cellSize = 3; } message CreateErasureCodingZoneResponseProto { @@ -81,4 +83,5 @@ message BlockECRecoveryInfoProto { required StorageTypesProto targetStorageTypes = 5; repeated uint32 liveBlockIndices = 6; required ECSchemaProto ecSchema = 7; + required uint32 cellSize = 8; } \ No newline at end of file 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 64030bebd37f0..f64cf8f025eb2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -359,7 +359,8 @@ message HdfsFileStatusProto { // Optional field for erasure coding optional ECSchemaProto ecSchema = 17; -} + optional uint32 stripeCellSize = 18; +} /** * Checksum algorithms/types used in HDFS 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 9f106cfcabe92..558c45d8c3736 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 @@ -1867,7 +1867,7 @@ public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir assert dir != null; dfs.mkdirs(dir); try { - dfs.getClient().createErasureCodingZone(dir.toString(), null); + dfs.getClient().createErasureCodingZone(dir.toString(), null, 0); } catch (IOException e) { if (!e.getMessage().contains("non-empty directory")) { throw e; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java index ec88a542de45b..4918dee4c41f2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java @@ -255,12 +255,12 @@ public Object answer(InvocationOnMock invocation) Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null, 0)).when(mockNN).getFileInfo(anyString()); Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null)) + 1010, 0, null, (byte) 0, null, 0)) .when(mockNN) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 9032d09d68696..a71441f021d20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -73,7 +73,7 @@ public void setup() throws IOException { cluster.waitActive(); fs = cluster.getFileSystem(); fs.mkdirs(dirPath); - fs.getClient().createErasureCodingZone(dirPath.toString(), null); + fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE); } @After @@ -94,7 +94,7 @@ public void testGetBlock() throws Exception { LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema); + filePath.toString(), false, schema, CELLSIZE); List lbList = lbs.getLocatedBlocks(); for (LocatedBlock aLbList : lbList) { @@ -146,7 +146,7 @@ public void testPread() throws Exception { } } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema); + filePath.toString(), false, schema, CELLSIZE); int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102, CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102, @@ -188,7 +188,7 @@ public void testPreadWithDNFailure() throws Exception { } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, - ErasureCodingSchemaManager.getSystemDefaultSchema()); + ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE); int readSize = BLOCK_GROUP_SIZE; byte[] readBuffer = new byte[readSize]; byte[] expected = new byte[readSize]; @@ -284,7 +284,7 @@ private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), - false, schema); + false, schema, CELLSIZE); byte[] expected = new byte[fileSize]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index ec98e68612f61..e795ea7a8edc1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -65,7 +65,7 @@ public void setup() throws IOException { Configuration conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index c2e588ad5012d..4ad3b2e4e0a47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -74,7 +74,7 @@ public void setup() throws IOException { cluster.waitActive(); dfs = cluster.getFileSystem(); dfs.mkdirs(dir); - dfs.createErasureCodingZone(dir, null); + dfs.createErasureCodingZone(dir, null, 0); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java index 6e2ec5ec95354..1f317b85998c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java @@ -737,7 +737,7 @@ private static void mockCreate(ClientProtocol mcp, version, new byte[suite.getAlgorithmBlockSize()], new byte[suite.getAlgorithmBlockSize()], "fakeKey", "fakeVersion"), - (byte) 0, null)) + (byte) 0, null, 0)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index 59818c09c33a8..a7c3cd4d45fbf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -63,7 +63,7 @@ public void testCreateECZone() fs.mkdir(testDir, FsPermission.getDirDefault()); /* Normal creation of an erasure coding zone */ - fs.getClient().createErasureCodingZone(testDir.toString(), null); + fs.getClient().createErasureCodingZone(testDir.toString(), null, 0); /* Verify files under the zone are striped */ final Path ECFilePath = new Path(testDir, "foo"); @@ -76,7 +76,7 @@ public void testCreateECZone() fs.mkdir(notEmpty, FsPermission.getDirDefault()); fs.create(new Path(notEmpty, "foo")); try { - fs.getClient().createErasureCodingZone(notEmpty.toString(), null); + fs.getClient().createErasureCodingZone(notEmpty.toString(), null, 0); fail("Erasure coding zone on non-empty dir"); } catch (IOException e) { assertExceptionContains("erasure coding zone for a non-empty directory", e); @@ -86,10 +86,10 @@ public void testCreateECZone() final Path zone1 = new Path("/zone1"); final Path zone2 = new Path(zone1, "zone2"); fs.mkdir(zone1, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(zone1.toString(), null); + fs.getClient().createErasureCodingZone(zone1.toString(), null, 0); fs.mkdir(zone2, FsPermission.getDirDefault()); try { - fs.getClient().createErasureCodingZone(zone2.toString(), null); + fs.getClient().createErasureCodingZone(zone2.toString(), null, 0); fail("Nested erasure coding zones"); } catch (IOException e) { assertExceptionContains("already in an erasure coding zone", e); @@ -99,7 +99,7 @@ public void testCreateECZone() final Path fPath = new Path("/file"); fs.create(fPath); try { - fs.getClient().createErasureCodingZone(fPath.toString(), null); + fs.getClient().createErasureCodingZone(fPath.toString(), null, 0); fail("Erasure coding zone on file"); } catch (IOException e) { assertExceptionContains("erasure coding zone for a file", e); @@ -112,8 +112,8 @@ public void testMoveValidity() throws IOException, InterruptedException { final Path dstECDir = new Path("/dstEC"); fs.mkdir(srcECDir, FsPermission.getDirDefault()); fs.mkdir(dstECDir, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(srcECDir.toString(), null); - fs.getClient().createErasureCodingZone(dstECDir.toString(), null); + fs.getClient().createErasureCodingZone(srcECDir.toString(), null, 0); + fs.getClient().createErasureCodingZone(dstECDir.toString(), null, 0); final Path srcFile = new Path(srcECDir, "foo"); fs.create(srcFile); @@ -157,7 +157,7 @@ public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { // dir ECInfo before creating ec zone assertNull(fs.getClient().getErasureCodingInfo(src)); // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, null); //Default one will be used. + fs.getClient().createErasureCodingZone(src, null, 0); //Default one will be used. ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); verifyErasureCodingInfo(src, sysDefaultSchema); fs.create(new Path(ecDir, "/child1")).close(); @@ -178,7 +178,7 @@ public void testGetErasureCodingInfo() throws Exception { // dir ECInfo before creating ec zone assertNull(fs.getClient().getErasureCodingInfo(src)); // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, usingSchema); + fs.getClient().createErasureCodingZone(src, usingSchema, 0); verifyErasureCodingInfo(src, usingSchema); fs.create(new Path(ecDir, "/child1")).close(); // verify for the files in ec zone diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java index f8c0667eb00e7..3c400b7a9563c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java @@ -50,7 +50,7 @@ public void testFileStatusWithECschema() throws Exception { final ECSchema schema1 = ErasureCodingSchemaManager.getSystemDefaultSchema(); // create EC zone on dir - fs.createErasureCodingZone(dir, schema1); + fs.createErasureCodingZone(dir, schema1, 0); final ECSchema schame2 = client.getFileInfo(dir.toUri().getPath()).getECSchema(); assertNotNull(schame2); assertTrue(schema1.equals(schame2)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java index b77ff3a000668..d0cd3350963f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java @@ -354,12 +354,12 @@ public void testFactory() throws Exception { Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null, 0)).when(mcp).getFileInfo(anyString()); Mockito .doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null)) + 1010, 0, null, (byte) 0, null, 0)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index dfdcee29f6ca5..e5e324cfbbbb6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -77,7 +77,7 @@ public void setup() throws IOException { cluster.waitActive(); fs = cluster.getFileSystem(); - fs.getClient().createErasureCodingZone("/", null); + fs.getClient().createErasureCodingZone("/", null, 0); List datanodes = cluster.getDataNodes(); for (int i = 0; i < dnNum; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index f78fb7a5c0cc8..1976dcaa86627 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -59,7 +59,7 @@ public static void setup() throws IOException { conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null, cellSize); fs = cluster.getFileSystem(); } 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 1be00b6fd8142..a0b203887e862 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 @@ -663,7 +663,8 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices0 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo( new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0, - liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema()); + liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema(), + 64 * 1024); DatanodeInfo[] dnInfos1 = new DatanodeInfo[] { DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() }; DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil @@ -677,7 +678,8 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices1 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo( new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1, - liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema()); + liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema(), + 64 * 1024); List blkRecoveryInfosList = new ArrayList(); blkRecoveryInfosList.add(blkECRecoveryInfo0); blkRecoveryInfosList.add(blkECRecoveryInfo1); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index ba763604a83c0..a35cbf480a0b0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -74,7 +74,7 @@ public void setup() throws IOException { .numDataNodes(GROUP_SIZE).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); - dfs.getClient().createErasureCodingZone("/", null); + dfs.getClient().createErasureCodingZone("/", null, 0); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 290942372d44c..abb9bf5aea796 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -451,7 +451,7 @@ public void testAddNewStripedBlock() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir, null); + fs.getClient().getNamenode().createErasureCodingZone(testDir, null, 0); // Create a file with striped block Path p = new Path(testFilePath); @@ -523,7 +523,7 @@ public void testUpdateStripedBlocks() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir, null); + fs.getClient().getNamenode().createErasureCodingZone(testDir, null, 0); //create a file with striped blocks Path p = new Path(testFilePath); 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 5e015bf0c3696..8fd0753448e8d 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 @@ -140,7 +140,7 @@ private void testPersistHelper(Configuration conf) throws IOException { private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf, boolean isUC) throws IOException{ // contruct a INode with StripedBlock for saving and loading - fsn.createErasureCodingZone("/", null, false); + fsn.createErasureCodingZone("/", null, 0, false); long id = 123456789; byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes(); PermissionStatus permissionStatus = new PermissionStatus("testuser_a", @@ -402,7 +402,7 @@ public void testSupportBlockGroup() throws IOException { .build(); cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); - fs.getClient().getNamenode().createErasureCodingZone("/", null); + fs.getClient().getNamenode().createErasureCodingZone("/", null, 0); Path file = new Path("/striped"); FSDataOutputStream out = fs.create(file); byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE); 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 787082791a6ca..b36beb2a5887b 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 @@ -1198,7 +1198,7 @@ public void testFsckFileNotFound() throws Exception { HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication, blockSize, modTime, accessTime, perms, owner, group, symlink, - path, fileId, numChildren, null, storagePolicy, null); + path, fileId, numChildren, null, storagePolicy, null, 0); Result res = new Result(conf); try { @@ -1629,4 +1629,4 @@ public void testFsckWithDecommissionedReplicas() throws Exception { } } } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java index 86fcb88bd9a53..9c585a4a2380c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java @@ -66,7 +66,7 @@ public void setUp() throws IOException { dfs = cluster.getFileSystem(); dfs.mkdirs(ecDir); - dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema); + dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema, 0); dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA); dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA); dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java index f3ef39a80c19d..2a51f99b79e9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java @@ -61,7 +61,7 @@ public static void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.waitActive(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0); fs = cluster.getFileSystem(); Path eczone = new Path("/eczone"); fs.mkdirs(eczone); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java index 6f29d6904831d..5a1c3fc096598 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -152,7 +152,7 @@ private byte[][] createInternalBlkBuffers(int bgSize) { int done = 0; while (done < bgSize) { Preconditions.checkState(done % CELLSIZE == 0); - StripingCell cell = new StripingCell(SCEHMA, done / CELLSIZE); + StripingCell cell = new StripingCell(SCEHMA, CELLSIZE, done / CELLSIZE); int idxInStripe = cell.idxInStripe; int size = Math.min(CELLSIZE, bgSize - done); for (int i = 0; i < size; i++) { @@ -247,7 +247,7 @@ public void testDivideByteRangeIntoStripes() { continue; } AlignedStripe[] stripes = divideByteRangeIntoStripes(SCEHMA, - blockGroup, brStart, brStart + brSize - 1, assembled, 0); + CELLSIZE, blockGroup, brStart, brStart + brSize - 1, assembled, 0); for (AlignedStripe stripe : stripes) { for (int i = 0; i < DATA_BLK_NUM; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java index 8947c5b49a16d..303d063ecc65a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java @@ -65,7 +65,7 @@ public void testHdfsFileStatus() throws IOException { final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26, now, now + 10, new FsPermission((short) 0644), "user", "group", DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"), - HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null); + HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null, 0); final FileStatus fstatus = toFileStatus(status, parent); System.out.println("status = " + status); System.out.println("fstatus = " + fstatus); From 47ef869fa790dd096b576697c4245d2f3a3193fa Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 19 May 2015 17:25:27 -0700 Subject: [PATCH 127/212] HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. Contributed by Yi Liu. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 9 ++++++--- .../hadoop/hdfs/server/blockmanagement/BlockManager.java | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index d6c9dba6958e7..48bc9d6919f6d 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -59,7 +59,7 @@ HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (vinayakumarb) - + HDFS-8074. Define a system-wide default EC schema. (Kai Zheng) HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. @@ -110,7 +110,7 @@ HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader. (szetszwo via Zhe Zhang) - + HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver. (szetszwo via Zhe Zhang) @@ -172,7 +172,7 @@ HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via umamahesh) - + HDFS-7672. Handle write failure for stripping blocks and refactor the existing code in DFSStripedOutputStream and StripedDataStreamer. (szetszwo) @@ -235,3 +235,6 @@ (Rakesh R via waltersu4549) HDFS-8375. Add cellSize as an XAttr to ECZone. ( Vinayakumar B via zhz). + + HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. + (Yi Liu via zhz). 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 28e7b89e06910..fc1396580dcce 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 @@ -3396,7 +3396,7 @@ public void processIncrementalBlockReport(final DatanodeID nodeID, for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) { switch (rdbi.getStatus()) { case DELETED_BLOCK: - removeStoredBlock(storageInfo, getStoredBlock(rdbi.getBlock()), node); + removeStoredBlock(storageInfo, rdbi.getBlock(), node); deleted++; break; case RECEIVED_BLOCK: From 45db1a0b727965ceb27411ea5567b3a76c63b181 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 19 May 2015 21:19:51 -0700 Subject: [PATCH 128/212] HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. Contributed by Tsz Wo Nicholas Sze. --- .../hdfs/protocol/LocatedStripedBlock.java | 12 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedOutputStream.java | 106 ++++++++++-------- .../org/apache/hadoop/hdfs/DataStreamer.java | 15 ++- .../hadoop/hdfs/StripedDataStreamer.java | 67 ++++++++--- .../blockmanagement/DatanodeStorageInfo.java | 15 ++- .../hdfs/server/namenode/FSNamesystem.java | 38 ++++--- .../server/namenode/NameNodeRpcServer.java | 2 +- .../hadoop/hdfs/util/StripedBlockUtil.java | 40 ++++++- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 8 +- ...TestDFSStripedOutputStreamWithFailure.java | 69 ++++++++---- .../server/namenode/TestAddStripedBlocks.java | 12 +- 12 files changed, 258 insertions(+), 129 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java index 93a5948007cdc..dc5a77fab2271 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -31,15 +31,21 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class LocatedStripedBlock extends LocatedBlock { + private static final int[] EMPTY_INDICES = {}; + private int[] blockIndices; public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs, StorageType[] storageTypes, int[] indices, long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) { super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs); - assert indices != null && indices.length == locs.length; - this.blockIndices = new int[indices.length]; - System.arraycopy(indices, 0, blockIndices, 0, indices.length); + + if (indices == null) { + this.blockIndices = EMPTY_INDICES; + } else { + this.blockIndices = new int[indices.length]; + System.arraycopy(indices, 0, blockIndices, 0, indices.length); + } } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 48bc9d6919f6d..b608b10722b11 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -238,3 +238,6 @@ HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. (Yi Liu via zhz). + + HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. + (Tsz Wo Nicholas Sze via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 4399a37f85931..8eed6ada47ec3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -33,7 +33,6 @@ import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; -import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -61,52 +60,72 @@ @InterfaceAudience.Private public class DFSStripedOutputStream extends DFSOutputStream { - /** Coordinate the communication between the streamers. */ - static class Coordinator { - private final DfsClientConf conf; - private final List> endBlocks; - private final List> stripedBlocks; - private volatile boolean shouldLocateFollowingBlock = false; - - Coordinator(final DfsClientConf conf, final int numDataBlocks, - final int numAllBlocks) { - this.conf = conf; - endBlocks = new ArrayList<>(numDataBlocks); - for (int i = 0; i < numDataBlocks; i++) { - endBlocks.add(new LinkedBlockingQueue(1)); + static class MultipleBlockingQueue { + private final int pullTimeout; + private final List> queues; + + MultipleBlockingQueue(int numQueue, int queueSize, int pullTimeout) { + queues = new ArrayList<>(numQueue); + for (int i = 0; i < numQueue; i++) { + queues.add(new LinkedBlockingQueue(queueSize)); } - stripedBlocks = new ArrayList<>(numAllBlocks); - for (int i = 0; i < numAllBlocks; i++) { - stripedBlocks.add(new LinkedBlockingQueue(1)); + this.pullTimeout = pullTimeout; + } + + void offer(int i, T object) { + final boolean b = queues.get(i).offer(object); + Preconditions.checkState(b, "Failed to offer " + object + + " to queue, i=" + i); + } + + T poll(int i) throws InterruptedIOException { + try { + return queues.get(i).poll(pullTimeout, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw DFSUtil.toInterruptedIOException("poll interrupted, i=" + i, e); } } - boolean shouldLocateFollowingBlock() { - return shouldLocateFollowingBlock; + T peek(int i) { + return queues.get(i).peek(); } + } - void putEndBlock(int i, ExtendedBlock block) { - shouldLocateFollowingBlock = true; + /** Coordinate the communication between the streamers. */ + static class Coordinator { + private final MultipleBlockingQueue stripedBlocks; + private final MultipleBlockingQueue endBlocks; + private final MultipleBlockingQueue updateBlocks; - final boolean b = endBlocks.get(i).offer(block); - Preconditions.checkState(b, "Failed to add " + block - + " to endBlocks queue, i=" + i); + Coordinator(final DfsClientConf conf, final int numDataBlocks, + final int numAllBlocks) { + stripedBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1, + conf.getStripedWriteMaxSecondsGetStripedBlock()); + endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1, + conf.getStripedWriteMaxSecondsGetEndedBlock()); + updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1, + conf.getStripedWriteMaxSecondsGetStripedBlock()); + } + + void putEndBlock(int i, ExtendedBlock block) { + endBlocks.offer(i, block); } ExtendedBlock getEndBlock(int i) throws InterruptedIOException { - try { - return endBlocks.get(i).poll( - conf.getStripedWriteMaxSecondsGetEndedBlock(), - TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw DFSUtil.toInterruptedIOException( - "getEndBlock interrupted, i=" + i, e); - } + return endBlocks.poll(i); + } + + void putUpdateBlock(int i, ExtendedBlock block) { + updateBlocks.offer(i, block); + } + + ExtendedBlock getUpdateBlock(int i) throws InterruptedIOException { + return updateBlocks.poll(i); } void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) { - ExtendedBlock b = endBlocks.get(i).peek(); + ExtendedBlock b = endBlocks.peek(i); if (b == null) { // streamer just has failed, put end block and continue b = block; @@ -119,22 +138,11 @@ void putStripedBlock(int i, LocatedBlock block) throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("putStripedBlock " + block + ", i=" + i); } - final boolean b = stripedBlocks.get(i).offer(block); - if (!b) { - throw new IOException("Failed: " + block + ", i=" + i); - } + stripedBlocks.offer(i, block); } LocatedBlock getStripedBlock(int i) throws IOException { - final LocatedBlock lb; - try { - lb = stripedBlocks.get(i).poll( - conf.getStripedWriteMaxSecondsGetStripedBlock(), - TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e); - } - + final LocatedBlock lb = stripedBlocks.poll(i); if (lb == null) { throw new IOException("Failed: i=" + i); } @@ -218,6 +226,11 @@ private StripedDataStreamer getLeadingStreamer() { return streamers.get(0); } + @Override + ExtendedBlock getBlock() { + return getLeadingStreamer().getBlock(); + } + /** Construct a new output stream for creating a file. */ DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, EnumSet flag, Progressable progress, @@ -292,6 +305,7 @@ private void checkStreamers() throws IOException { int count = 0; for(StripedDataStreamer s : streamers) { if (!s.isFailed()) { + s.getErrorState().initExtenalError(); count++; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index 8f07341f749c2..1344d54d61a0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -44,7 +44,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.StorageType; -import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite; import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; @@ -210,6 +209,7 @@ synchronized void throwException4Close() throws IOException { static class ErrorState { private boolean error = false; + private boolean extenalError = false; private int badNodeIndex = -1; private int restartingNodeIndex = -1; private long restartingNodeDeadline = 0; @@ -221,6 +221,7 @@ static class ErrorState { synchronized void reset() { error = false; + extenalError = false; badNodeIndex = -1; restartingNodeIndex = -1; restartingNodeDeadline = 0; @@ -231,13 +232,19 @@ synchronized boolean hasError() { } synchronized boolean hasDatanodeError() { - return error && isNodeMarked(); + return error && (isNodeMarked() || extenalError); } synchronized void setError(boolean err) { this.error = err; } + synchronized void initExtenalError() { + setError(true); + this.extenalError = true; + } + + synchronized void setBadNodeIndex(int index) { this.badNodeIndex = index; } @@ -1736,6 +1743,10 @@ Token getBlockToken() { return accessToken; } + ErrorState getErrorState() { + return errorState; + } + /** * Put a packet to the data queue * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 258fc6505dd85..7b7db7532936f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -94,36 +94,69 @@ protected void endBlock() { protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) throws IOException { if (isLeadingStreamer()) { - if (coordinator.shouldLocateFollowingBlock()) { + if (block != null) { // set numByte for the previous block group long bytes = 0; for (int i = 0; i < NUM_DATA_BLOCKS; i++) { final ExtendedBlock b = coordinator.getEndBlock(i); - bytes += b == null ? 0 : b.getNumBytes(); + if (b != null) { + StripedBlockUtil.checkBlocks(block, i, b); + bytes += b.getNumBytes(); + } } block.setNumBytes(bytes); } - final LocatedStripedBlock lsb - = (LocatedStripedBlock)super.locateFollowingBlock(excludedNodes); - if (LOG.isDebugEnabled()) { - LOG.debug("Obtained block group " + lsb); - } - LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(lsb, - BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); - - assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) : - "Fail to get block group from namenode: blockGroupSize: " + - (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " + - blocks.length; - for (int i = 0; i < blocks.length; i++) { - coordinator.putStripedBlock(i, blocks[i]); - } + putLoactedBlocks(super.locateFollowingBlock(excludedNodes)); } return coordinator.getStripedBlock(index); } + void putLoactedBlocks(LocatedBlock lb) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Obtained block group " + lb); + } + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock)lb, + BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + + // TODO allow write to continue if blocks.length >= NUM_DATA_BLOCKS + assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) : + "Fail to get block group from namenode: blockGroupSize: " + + (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " + + blocks.length; + for (int i = 0; i < blocks.length; i++) { + coordinator.putStripedBlock(i, blocks[i]); + } + } + + @Override + LocatedBlock updateBlockForPipeline() throws IOException { + if (isLeadingStreamer()) { + final LocatedBlock updated = super.updateBlockForPipeline(); + final ExtendedBlock block = updated.getBlock(); + for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { + final LocatedBlock lb = new LocatedBlock(block, null, null, null, + -1, updated.isCorrupt(), null); + lb.setBlockToken(updated.getBlockToken()); + coordinator.putStripedBlock(i, lb); + } + } + return coordinator.getStripedBlock(index); + } + + @Override + ExtendedBlock updatePipeline(long newGS) throws IOException { + if (isLeadingStreamer()) { + final ExtendedBlock newBlock = super.updatePipeline(newGS); + for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { + coordinator.putUpdateBlock(i, new ExtendedBlock(newBlock)); + } + } + return coordinator.getUpdateBlock(index); + } + @Override public String toString() { return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0) 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 index 9b5a923862bdd..2275d91bccefb 100644 --- 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 @@ -21,18 +21,15 @@ import java.util.Iterator; import java.util.List; -import com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage; -import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.LocatedBlock; 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; +import com.google.common.annotations.VisibleForTesting; + /** * A Datanode has one or more storages. A storage in the Datanode is represented * by this class. @@ -41,7 +38,7 @@ public class DatanodeStorageInfo { public static final DatanodeStorageInfo[] EMPTY_ARRAY = {}; public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) { - return toDatanodeInfos(Arrays.asList(storages)); + return storages == null? null: toDatanodeInfos(Arrays.asList(storages)); } static DatanodeInfo[] toDatanodeInfos(List storages) { final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()]; @@ -61,6 +58,9 @@ static DatanodeDescriptor[] toDatanodeDescriptors( } public static String[] toStorageIDs(DatanodeStorageInfo[] storages) { + if (storages == null) { + return null; + } String[] storageIDs = new String[storages.length]; for(int i = 0; i < storageIDs.length; i++) { storageIDs[i] = storages[i].getStorageID(); @@ -69,6 +69,9 @@ public static String[] toStorageIDs(DatanodeStorageInfo[] storages) { } public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) { + if (storages == null) { + return null; + } StorageType[] storageTypes = new StorageType[storages.length]; for(int i = 0; i < storageTypes.length; i++) { storageTypes[i] = storages[i].getStorageType(); 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 60f86d6fa3274..7e45e9076c7f1 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 @@ -5787,29 +5787,30 @@ void reportBadBlocks(LocatedBlock[] blocks) throws IOException { * Get a new generation stamp together with an access token for * a block under construction * - * This method is called for recovering a failed pipeline or setting up - * a pipeline to append to a block. + * This method is called for recovering a failed write or setting up + * a block for appended. * * @param block a block * @param clientName the name of a client * @return a located block with a new generation stamp and an access token * @throws IOException if any error occurs */ - LocatedBlock updateBlockForPipeline(ExtendedBlock block, + LocatedBlock bumpBlockGenerationStamp(ExtendedBlock block, String clientName) throws IOException { - LocatedBlock locatedBlock; + final LocatedBlock locatedBlock; checkOperation(OperationCategory.WRITE); writeLock(); try { checkOperation(OperationCategory.WRITE); // check vadility of parameters - checkUCBlock(block, clientName); + final INodeFile file = checkUCBlock(block, clientName); // get a new generation stamp and an access token block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock()))); - locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]); - blockManager.setBlockToken(locatedBlock, BlockTokenIdentifier.AccessMode.WRITE); + + locatedBlock = BlockManager.newLocatedBlock( + block, file.getLastBlock(), null, -1); } finally { writeUnlock(); } @@ -5864,23 +5865,26 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, // check the vadility of the block and lease holder name final INodeFile pendingFile = checkUCBlock(oldBlock, clientName); final BlockInfo lastBlock = pendingFile.getLastBlock(); - // when updating pipeline, the last block must be contiguous block - assert lastBlock instanceof BlockInfoContiguousUnderConstruction; - BlockInfoContiguousUnderConstruction blockinfo = - (BlockInfoContiguousUnderConstruction) lastBlock; + final BlockInfoUnderConstruction blockinfo = (BlockInfoUnderConstruction)lastBlock; // check new GS & length: this is not expected - if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() || - newBlock.getNumBytes() < blockinfo.getNumBytes()) { - String msg = "Update " + oldBlock + " (len = " + - blockinfo.getNumBytes() + ") to an older state: " + newBlock + - " (len = " + newBlock.getNumBytes() +")"; + if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) { + final String msg = "Update " + oldBlock + " but the new block " + newBlock + + " does not have a larger generation stamp than the last block " + + lastBlock; + LOG.warn(msg); + throw new IOException(msg); + } + if (newBlock.getNumBytes() < lastBlock.getNumBytes()) { + final String msg = "Update " + oldBlock + " (size=" + + oldBlock.getNumBytes() + ") to a smaller size block " + newBlock + + " (size=" + newBlock.getNumBytes() + ")"; LOG.warn(msg); throw new IOException(msg); } // Update old block with the new generation stamp and new length - blockinfo.setNumBytes(newBlock.getNumBytes()); + lastBlock.setNumBytes(newBlock.getNumBytes()); blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); // find the DatanodeDescriptor objects 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 747f5283d7710..9e94b90863e9e 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 @@ -788,7 +788,7 @@ public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName) throws IOException { checkNNStartup(); - return namesystem.updateBlockForPipeline(block, clientName); + return namesystem.bumpBlockGenerationStamp(block, clientName); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 6f7dcb1478685..0b09f3776c5b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -34,6 +34,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import java.util.*; +import java.io.IOException; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; @@ -104,12 +105,17 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, final ExtendedBlock blk = constructInternalBlock( bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); - return new LocatedBlock(blk, - new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, - new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, - new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, - bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), - null); + final long offset = bg.getStartOffset() + idxInBlockGroup * cellSize; + if (idxInReturnedLocs < bg.getLocations().length) { + return new LocatedBlock(blk, + new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, + new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, + new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, + offset, bg.isCorrupt(), null); + } else { + return new LocatedBlock(blk, null, null, null, + offset, bg.isCorrupt(), null); + } } /** @@ -823,4 +829,26 @@ public String toString() { return "(index=" + index + ", state =" + state + ")"; } } + + /** + * Check if the information such as IDs and generation stamps in block-i + * match block-0. + */ + public static void checkBlocks(ExtendedBlock block0, int i, + ExtendedBlock blocki) throws IOException { + + if (!blocki.getBlockPoolId().equals(block0.getBlockPoolId())) { + throw new IOException("Block pool IDs mismatched: block0=" + + block0 + ", block" + i + "=" + blocki); + } + if (blocki.getBlockId() - i != block0.getBlockId()) { + throw new IOException("Block IDs mismatched: block0=" + + block0 + ", block" + i + "=" + blocki); + } + if (blocki.getGenerationStamp() != block0.getGenerationStamp()) { + throw new IOException("Generation stamps mismatched: block0=" + + block0 + ", block" + i + "=" + blocki); + } + } + } 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 558c45d8c3736..82c078141fb2a 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 @@ -1953,11 +1953,9 @@ public static Block addStripedBlockToFile(List dataNodes, * Because currently DFSStripedOutputStream does not support hflush/hsync, * tests can use this method to flush all the buffered data to DataNodes. */ - public static void writeAndFlushStripedOutputStream( - DFSStripedOutputStream out, int chunkSize) throws IOException { - // FSOutputSummer.BUFFER_NUM_CHUNKS == 9 - byte[] toWrite = new byte[chunkSize * 9 + 1]; - out.write(toWrite); + public static ExtendedBlock flushInternal(DFSStripedOutputStream out) + throws IOException { out.flushInternal(); + return out.getBlock(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index 4ad3b2e4e0a47..c232e13610180 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; @@ -44,6 +43,8 @@ import org.junit.Before; import org.junit.Test; +import com.google.common.base.Preconditions; + public class TestDFSStripedOutputStreamWithFailure { public static final Log LOG = LogFactory.getLog( TestDFSStripedOutputStreamWithFailure.class); @@ -59,6 +60,9 @@ public class TestDFSStripedOutputStreamWithFailure { private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK; private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS; + private static final int FLUSH_POS + = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1; + private final HdfsConfiguration conf = new HdfsConfiguration(); private MiniDFSCluster cluster; private DistributedFileSystem dfs; @@ -149,50 +153,53 @@ private void runTest(final String src, final int length, final int dnIndex) { cluster.startDataNodes(conf, 1, true, null, null); cluster.waitActive(); - runTest(new Path(dir, src), length, dnIndex); + runTest(new Path(dir, src), length, length/2, dnIndex); } catch(Exception e) { LOG.info("FAILED", e); Assert.fail(StringUtils.stringifyException(e)); } } - private void runTest(final Path p, final int length, + private void runTest(final Path p, final int length, final int killPos, final int dnIndex) throws Exception { - LOG.info("p=" + p + ", length=" + length + ", dnIndex=" + dnIndex); + LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos + + ", dnIndex=" + dnIndex); + Preconditions.checkArgument(killPos < length); + Preconditions.checkArgument(killPos > FLUSH_POS); final String fullPath = p.toString(); final AtomicInteger pos = new AtomicInteger(); final FSDataOutputStream out = dfs.create(p); - final AtomicBoolean killed = new AtomicBoolean(); - final Thread killer = new Thread(new Runnable() { - @Override - public void run() { - killDatanode(cluster, (DFSStripedOutputStream)out.getWrappedStream(), - dnIndex, pos); - killed.set(true); - } - }); - killer.start(); + final DFSStripedOutputStream stripedOut + = (DFSStripedOutputStream)out.getWrappedStream(); - final int mask = (1 << 16) - 1; + long oldGS = -1; + boolean killed = false; for(; pos.get() < length; ) { final int i = pos.getAndIncrement(); + if (i == killPos) { + final long gs = getGenerationStamp(stripedOut); + Assert.assertTrue(oldGS != -1); + Assert.assertEquals(oldGS, gs); + + killDatanode(cluster, stripedOut, dnIndex, pos); + killed = true; + } + write(out, i); - if ((i & mask) == 0) { - final long ms = 100; - LOG.info("i=" + i + " sleep " + ms); - Thread.sleep(ms); + + if (i == FLUSH_POS) { + oldGS = getGenerationStamp(stripedOut); } } - killer.join(10000); - Assert.assertTrue(killed.get()); out.close(); + Assert.assertTrue(killed); // check file length final FileStatus status = dfs.getFileStatus(p); Assert.assertEquals(length, status.getLen()); - checkData(dfs, fullPath, length, dnIndex); + checkData(dfs, fullPath, length, dnIndex, oldGS); } static void write(FSDataOutputStream out, int i) throws IOException { @@ -203,6 +210,14 @@ static void write(FSDataOutputStream out, int i) throws IOException { } } + static long getGenerationStamp(DFSStripedOutputStream out) + throws IOException { + final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp(); + LOG.info("getGenerationStamp returns " + gs); + return gs; + + } + static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { for(;;) { final DatanodeInfo[] datanodes = streamer.getNodes(); @@ -228,7 +243,7 @@ static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, } static void checkData(DistributedFileSystem dfs, String src, int length, - int killedDnIndex) throws IOException { + int killedDnIndex, long oldGS) throws IOException { List> blockGroupList = new ArrayList<>(); LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L); final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1; @@ -236,6 +251,12 @@ static void checkData(DistributedFileSystem dfs, String src, int length, for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); + + final long gs = firstBlock.getBlock().getGenerationStamp(); + final String s = "gs=" + gs + ", oldGS=" + oldGS; + LOG.info(s); + Assert.assertTrue(s, gs > oldGS); + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( (LocatedStripedBlock) firstBlock, CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); @@ -247,7 +268,7 @@ static void checkData(DistributedFileSystem dfs, String src, int length, final boolean isLastGroup = group == blockGroupList.size() - 1; final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE; - final int numCellInGroup = (int)((groupSize - 1)/CELL_SIZE + 1); + final int numCellInGroup = (groupSize - 1)/CELL_SIZE + 1; final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS; final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index a35cbf480a0b0..7876d1a1d2f90 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -105,6 +105,14 @@ public void testAllocateBlockId() throws Exception { Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId); } + private static void writeAndFlushStripedOutputStream( + DFSStripedOutputStream out, int chunkSize) throws IOException { + // FSOutputSummer.BUFFER_NUM_CHUNKS == 9 + byte[] toWrite = new byte[chunkSize * 9 + 1]; + out.write(toWrite); + DFSTestUtil.flushInternal(out); + } + @Test (timeout=60000) public void testAddStripedBlock() throws Exception { final Path file = new Path("/file1"); @@ -112,7 +120,7 @@ public void testAddStripedBlock() throws Exception { FSDataOutputStream out = null; try { out = dfs.create(file, (short) 1); - DFSTestUtil.writeAndFlushStripedOutputStream( + writeAndFlushStripedOutputStream( (DFSStripedOutputStream) out.getWrappedStream(), DFS_BYTES_PER_CHECKSUM_DEFAULT); @@ -190,7 +198,7 @@ public void testGetLocatedStripedBlocks() throws Exception { FSDataOutputStream out = null; try { out = dfs.create(file, (short) 1); - DFSTestUtil.writeAndFlushStripedOutputStream( + writeAndFlushStripedOutputStream( (DFSStripedOutputStream) out.getWrappedStream(), DFS_BYTES_PER_CHECKSUM_DEFAULT); From 744ef1779202621b2034b97926ad7ebcc67f433a Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 20 May 2015 14:52:51 -0700 Subject: [PATCH 129/212] HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped. Contributed by Kai Sasaki. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../server/blockmanagement/BlockInfoStriped.java | 15 ++++++--------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index b608b10722b11..610a5eb103be6 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -241,3 +241,6 @@ HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. (Tsz Wo Nicholas Sze via jing9) + + HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped. + (Kai Sasaki via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index d7a48a0b56420..3898a495ada83 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -19,7 +19,6 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -39,8 +38,6 @@ * array to record the block index for each triplet. */ public class BlockInfoStriped extends BlockInfo { - private final short dataBlockNum; - private final short parityBlockNum; private final ECSchema schema; /** * Always the same size with triplets. Record the block index for each triplet @@ -54,8 +51,6 @@ public BlockInfoStriped(Block blk, ECSchema schema) { indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()]; initIndices(); this.schema = schema; - this.dataBlockNum = (short)schema.getNumDataUnits(); - this.parityBlockNum = (short)schema.getNumParityUnits(); } BlockInfoStriped(BlockInfoStriped b) { @@ -64,15 +59,16 @@ public BlockInfoStriped(Block blk, ECSchema schema) { } public short getTotalBlockNum() { - return (short) (dataBlockNum + parityBlockNum); + return (short) (this.schema.getNumDataUnits() + + this.schema.getNumParityUnits()); } public short getDataBlockNum() { - return dataBlockNum; + return (short) this.schema.getNumDataUnits(); } public short getParityBlockNum() { - return parityBlockNum; + return (short) this.schema.getNumParityUnits(); } public ECSchema getSchema() { @@ -210,7 +206,8 @@ public long spaceConsumed() { // be the total of data blocks and parity blocks because // `getNumBytes` is the total of actual data block size. return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(), - dataBlockNum, parityBlockNum, BLOCK_STRIPED_CELL_SIZE); + this.schema.getNumDataUnits(), this.schema.getNumParityUnits(), + BLOCK_STRIPED_CELL_SIZE); } @Override From e53fa769c97416af69ea567aecd44f67e896688b Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 20 May 2015 15:37:50 -0700 Subject: [PATCH 130/212] HDFS-8186. Erasure coding: Make block placement policy for EC file configurable. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 3 ++ .../hadoop/hdfs/server/balancer/Balancer.java | 6 ++- .../server/blockmanagement/BlockManager.java | 32 ++++++----- .../BlockPlacementPolicies.java | 54 +++++++++++++++++++ .../blockmanagement/BlockPlacementPolicy.java | 26 +-------- .../server/namenode/FSDirWriteFileOp.java | 12 +++-- .../hdfs/server/namenode/FSNamesystem.java | 4 +- .../hdfs/server/namenode/NamenodeFsck.java | 9 ++-- 9 files changed, 95 insertions(+), 54 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 610a5eb103be6..3bdff6febeec3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -244,3 +244,6 @@ HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped. (Kai Sasaki via jing9) + + HDFS-8186. Erasure coding: Make block placement policy for EC file configurable. + (Walter Su via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index d5d30955b5b85..8e6b9f0b936a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolarent; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker; import org.apache.hadoop.http.HttpConfig; @@ -434,6 +435,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final Class DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class; public static final String DFS_REPLICATION_MAX_KEY = "dfs.replication.max"; public static final int DFS_REPLICATION_MAX_DEFAULT = 512; + public static final String DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY = "dfs.block.placement.ec.classname"; + public static final Class DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT = BlockPlacementPolicyRackFaultTolarent.class; public static final String DFS_DF_INTERVAL_KEY = "dfs.df.interval"; public static final int DFS_DF_INTERVAL_DEFAULT = 60000; 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 bc7e4489e0bc6..d756f2b5c82ce 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 @@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Source; import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Task; import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Util; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport; @@ -199,7 +199,9 @@ public class Balancer { */ private static void checkReplicationPolicyCompatibility(Configuration conf ) throws UnsupportedActionException { - if (!(BlockPlacementPolicy.getInstance(conf, null, null, null) instanceof + BlockPlacementPolicies placementPolicies = + new BlockPlacementPolicies(conf, null, null, null); + if (!(placementPolicies.getPolicy(false) instanceof BlockPlacementPolicyDefault)) { throw new UnsupportedActionException( "Balancer without BlockPlacementPolicyDefault"); 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 fc1396580dcce..79cbcc6e8db01 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 @@ -273,7 +273,7 @@ public int getPendingDataNodeMessageCount() { private double replicationQueuesInitProgress = 0.0; /** for block replicas placement */ - private BlockPlacementPolicy blockplacement; + private BlockPlacementPolicies placementPolicies; private final BlockStoragePolicySuite storagePolicySuite; /** Check whether name system is running before terminating */ @@ -297,7 +297,7 @@ public BlockManager(final Namesystem namesystem, final Configuration conf) // Compute the map capacity by allocating 2% of total memory blocksMap = new BlocksMap( LightWeightGSet.computeCapacity(2.0, "BlocksMap")); - blockplacement = BlockPlacementPolicy.getInstance( + placementPolicies = new BlockPlacementPolicies( conf, datanodeManager.getFSClusterStats(), datanodeManager.getNetworkTopology(), datanodeManager.getHost2DatanodeMap()); @@ -490,15 +490,7 @@ public DatanodeManager getDatanodeManager() { @VisibleForTesting public BlockPlacementPolicy getBlockPlacementPolicy() { - return blockplacement; - } - - /** Set BlockPlacementPolicy */ - public void setBlockPlacementPolicy(BlockPlacementPolicy newpolicy) { - if (newpolicy == null) { - throw new HadoopIllegalArgumentException("newpolicy == null"); - } - this.blockplacement = newpolicy; + return placementPolicies.getPolicy(false); } /** Dump meta data to out. */ @@ -1504,7 +1496,9 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { // choose replication targets: NOT HOLDING THE GLOBAL LOCK // It is costly to extract the filename for which chooseTargets is called, // so for now we pass in the block collection itself. - rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes); + final BlockPlacementPolicy placementPolicy = + placementPolicies.getPolicy(rw.block.isStriped()); + rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes); } // Step 3: add tasks to the DN @@ -1630,7 +1624,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { /** Choose target for WebHDFS redirection. */ public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src, DatanodeDescriptor clientnode, Set excludes, long blocksize) { - return blockplacement.chooseTarget(src, 1, clientnode, + return placementPolicies.getPolicy(false).chooseTarget(src, 1, clientnode, Collections.emptyList(), false, excludes, blocksize, storagePolicySuite.getDefaultPolicy()); } @@ -1642,9 +1636,10 @@ public DatanodeStorageInfo[] chooseTarget4AdditionalDatanode(String src, List chosen, Set excludes, long blocksize, - byte storagePolicyID) { - + byte storagePolicyID, + boolean isStriped) { final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID); + final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped); return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode, chosen, true, excludes, blocksize, storagePolicy); } @@ -1662,10 +1657,12 @@ public DatanodeStorageInfo[] chooseTarget4NewBlock(final String src, final Set excludedNodes, final long blocksize, final List favoredNodes, - final byte storagePolicyID) throws IOException { + final byte storagePolicyID, + final boolean isStriped) throws IOException { List favoredDatanodeDescriptors = getDatanodeDescriptors(favoredNodes); final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID); + final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped); final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src, numOfReplicas, client, excludedNodes, blocksize, favoredDatanodeDescriptors, storagePolicy); @@ -3088,7 +3085,7 @@ private void processOverReplicatedBlock(final BlockInfo block, } } chooseExcessReplicates(nonExcess, block, replication, - addedNode, delNodeHint, blockplacement); + addedNode, delNodeHint, placementPolicies.getPolicy(false)); } @@ -4126,4 +4123,5 @@ public void clear() { clearQueues(); blocksMap.clear(); } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java new file mode 100644 index 0000000000000..622b2581fdfae --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java @@ -0,0 +1,54 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.net.NetworkTopology; +import org.apache.hadoop.util.ReflectionUtils; + +public class BlockPlacementPolicies{ + + private final BlockPlacementPolicy replicationPolicy; + private final BlockPlacementPolicy ecPolicy; + + public BlockPlacementPolicies(Configuration conf, FSClusterStats stats, + NetworkTopology clusterMap, + Host2NodesMap host2datanodeMap){ + final Class replicatorClass = conf + .getClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, + DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT, + BlockPlacementPolicy.class); + replicationPolicy = ReflectionUtils.newInstance(replicatorClass, conf); + replicationPolicy.initialize(conf, stats, clusterMap, host2datanodeMap); + final Class blockPlacementECClass = + conf.getClass(DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, + DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT, + BlockPlacementPolicy.class); + ecPolicy = ReflectionUtils.newInstance(blockPlacementECClass, conf); + ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap); + } + + public BlockPlacementPolicy getPolicy(boolean isStriped){ + if (isStriped) { + return ecPolicy; + } else { + return replicationPolicy; + } + } +} 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 9696179185ed4..86aaf79cedcdc 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 @@ -145,31 +145,7 @@ abstract public DatanodeStorageInfo chooseReplicaToDelete( abstract protected void initialize(Configuration conf, FSClusterStats stats, NetworkTopology clusterMap, Host2NodesMap host2datanodeMap); - - /** - * Get an instance of the configured Block Placement Policy based on the - * the configuration property - * {@link DFSConfigKeys#DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}. - * - * @param conf the configuration to be used - * @param stats an object that is used to retrieve the load on the cluster - * @param clusterMap the network topology of the cluster - * @return an instance of BlockPlacementPolicy - */ - public static BlockPlacementPolicy getInstance(Configuration conf, - FSClusterStats stats, - NetworkTopology clusterMap, - Host2NodesMap host2datanodeMap) { - final Class replicatorClass = conf.getClass( - DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, - DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT, - BlockPlacementPolicy.class); - final BlockPlacementPolicy replicator = ReflectionUtils.newInstance( - replicatorClass, conf); - replicator.initialize(conf, stats, clusterMap, host2datanodeMap); - return replicator; - } - + /** * Adjust rackmap, moreThanOne, and exactlyOne after removing replica on cur. * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 6137fd51a5c5a..11b6d8f2f4f0a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -174,6 +174,7 @@ static ValidateAddBlockResult validateAddBlock( final short numTargets; final byte storagePolicyID; String clientMachine; + final boolean isStriped; byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); src = fsn.dir.resolvePath(pc, src, pathComponents); @@ -199,13 +200,13 @@ static ValidateAddBlockResult validateAddBlock( blockSize = pendingFile.getPreferredBlockSize(); clientMachine = pendingFile.getFileUnderConstructionFeature() .getClientMachine(); - boolean isStriped = pendingFile.isStriped(); + isStriped = pendingFile.isStriped(); numTargets = isStriped ? HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : pendingFile.getFileReplication(); storagePolicyID = pendingFile.getStoragePolicyID(); return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID, - clientMachine); + clientMachine, isStriped); } static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk, @@ -289,7 +290,8 @@ static DatanodeStorageInfo[] chooseTargetForNewBlock( // choose targets for the new block to be allocated. return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode, excludedNodesSet, r.blockSize, - favoredNodesList, r.storagePolicyID); + favoredNodesList, r.storagePolicyID, + r.isStriped); } /** @@ -867,14 +869,16 @@ static class ValidateAddBlockResult { final int numTargets; final byte storagePolicyID; final String clientMachine; + final boolean isStriped; ValidateAddBlockResult( long blockSize, int numTargets, byte storagePolicyID, - String clientMachine) { + String clientMachine, boolean isStriped) { this.blockSize = blockSize; this.numTargets = numTargets; this.storagePolicyID = storagePolicyID; this.clientMachine = clientMachine; + this.isStriped = isStriped; } } 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 7e45e9076c7f1..e9bb2f79f03af 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 @@ -2935,6 +2935,7 @@ LocatedBlock getAdditionalDatanode(String src, long fileId, final long preferredblocksize; final byte storagePolicyID; final List chosen; + final boolean isStriped; checkOperation(OperationCategory.READ); byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); FSPermissionChecker pc = getPermissionChecker(); @@ -2961,6 +2962,7 @@ LocatedBlock getAdditionalDatanode(String src, long fileId, clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine); preferredblocksize = file.getPreferredBlockSize(); storagePolicyID = file.getStoragePolicyID(); + isStriped = file.isStriped(); //find datanode storages final DatanodeManager dm = blockManager.getDatanodeManager(); @@ -2976,7 +2978,7 @@ LocatedBlock getAdditionalDatanode(String src, long fileId, // choose new datanodes. final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode( src, numAdditionalNodes, clientnode, chosen, - excludes, preferredblocksize, storagePolicyID); + excludes, preferredblocksize, storagePolicyID, isStriped); final LocatedBlock lb = BlockManager.newLocatedBlock( blk, targets, -1, false); blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY); 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 fccef1795cfdf..10673943dd96c 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 @@ -65,9 +65,8 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -171,7 +170,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { private final PrintWriter out; private List snapshottableDirs = null; - private final BlockPlacementPolicy bpPolicy; + private final BlockPlacementPolicies bpPolicies; private StoragePolicySummary storageTypeSummary = null; /** @@ -193,7 +192,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { this.out = out; this.totalDatanodes = totalDatanodes; this.remoteAddress = remoteAddress; - this.bpPolicy = BlockPlacementPolicy.getInstance(conf, null, + this.bpPolicies = new BlockPlacementPolicies(conf, null, networktopology, namenode.getNamesystem().getBlockManager().getDatanodeManager() .getHost2DatanodeMap()); @@ -601,7 +600,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res } // count mis replicated blocks - BlockPlacementStatus blockPlacementStatus = bpPolicy + BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(false) .verifyBlockPlacement(path, lBlk, targetFileReplication); if (!blockPlacementStatus.isPlacementPolicySatisfied()) { res.numMisReplicatedBlocks++; From 7af05a3db4c731eca5e674b3d9e3b7abbf82ccd5 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 21 May 2015 14:40:14 -0700 Subject: [PATCH 131/212] HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding. Contributed by Rakesh R. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../apache/hadoop/hdfs/DFSStripedOutputStream.java | 12 ++++++------ .../BlockInfoStripedUnderConstruction.java | 3 +++ .../datanode/erasurecode/ErasureCodingWorker.java | 4 ++-- .../server/namenode/ErasureCodingZoneManager.java | 4 ++-- .../apache/hadoop/hdfs/util/StripedBlockUtil.java | 6 +++--- 6 files changed, 19 insertions(+), 13 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 3bdff6febeec3..c986f190c46ae 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -247,3 +247,6 @@ HDFS-8186. Erasure coding: Make block placement policy for EC file configurable. (Walter Su via zhz) + + HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding. + (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 8eed6ada47ec3..515ce0cdf7b2f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -276,11 +276,11 @@ int getCurrentIndex() { return getCurrentStreamer().getIndex(); } - StripedDataStreamer getCurrentStreamer() { + private synchronized StripedDataStreamer getCurrentStreamer() { return (StripedDataStreamer)streamer; } - private StripedDataStreamer setCurrentStreamer(int i) { + private synchronized StripedDataStreamer setCurrentStreamer(int i) { streamer = streamers.get(i); return getCurrentStreamer(); } @@ -344,8 +344,8 @@ private List generatePackets( int ckOff = 0; while (byteBuffer.remaining() > 0) { DFSPacket p = createPacket(packetSize, chunksPerPacket, - streamer.getBytesCurBlock(), - streamer.getAndIncCurrentSeqno(), false); + getCurrentStreamer().getBytesCurBlock(), + getCurrentStreamer().getAndIncCurrentSeqno(), false); int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum; int toWrite = byteBuffer.remaining() > maxBytesToPacket ? maxBytesToPacket: byteBuffer.remaining(); @@ -353,7 +353,7 @@ private List generatePackets( p.writeChecksum(checksumBuf, ckOff, ckLen); ckOff += ckLen; p.writeData(byteBuffer, toWrite); - streamer.incBytesCurBlock(toWrite); + getCurrentStreamer().incBytesCurBlock(toWrite); packets.add(p); } return packets; @@ -529,7 +529,7 @@ void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf if (!current.isFailed()) { try { for (DFSPacket p : generatePackets(buffer, checksumBuf)) { - streamer.waitAndQueuePacket(p); + getCurrentStreamer().waitAndQueuePacket(p); } endBlock(); } catch(Exception e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index 40f880f9d5200..76d79201b511b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -189,6 +189,9 @@ public void initializeBlockRecovery(long recoveryId) { NameNode.blockStateChangeLog.warn("BLOCK*" + " BlockInfoStripedUnderConstruction.initLeaseRecovery:" + " No blocks found, lease removed."); + // sets primary node index and return. + primaryNodeIndex = -1; + return; } boolean allLiveReplicasTriedAsPrimary = true; for (ReplicaUnderConstruction replica : replicas) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index d227de85cf2e0..ded51eb585634 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -251,7 +251,7 @@ private class ReconstructAndTransferBlock implements Runnable { private final long[] blockOffset4Targets; private final long[] seqNo4Targets; - private final int WRITE_PACKET_SIZE = 64 * 1024; + private final static int WRITE_PACKET_SIZE = 64 * 1024; private DataChecksum checksum; private int maxChunksPerPacket; private byte[] packetBuf; @@ -904,7 +904,7 @@ private int initTargetStreams(boolean[] targetsStatus) { } } - private class StripedReader { + private static class StripedReader { private final short index; private BlockReader blockReader; private ByteBuffer buffer; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 371b8acf55f92..89fecc612c321 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -92,8 +92,8 @@ ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { String schemaName = WritableUtils.readString(dIn); ECSchema schema = dir.getFSNamesystem().getECSchemaManager() .getSchema(schemaName); - return new ErasureCodingZoneInfo(inode.getFullPathName(), schema, - cellSize); + return new ErasureCodingZoneInfo(dir.getInode(inode.getId()) + .getFullPathName(), schema, cellSize); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 0b09f3776c5b9..38dc61aa33572 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -105,7 +105,7 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, final ExtendedBlock blk = constructInternalBlock( bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); - final long offset = bg.getStartOffset() + idxInBlockGroup * cellSize; + final long offset = bg.getStartOffset() + idxInBlockGroup * (long) cellSize; if (idxInReturnedLocs < bg.getLocations().length) { return new LocatedBlock(blk, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, @@ -406,11 +406,11 @@ public static long[] getStartOffsetsForInternalBlocks(ECSchema ecSchema, long earliestStart = startOffsets[firstCell.idxInStripe]; for (int i = 1; i < dataBlkNum; i++) { int idx = firstCellIdxInBG + i; - if (idx * cellSize >= blockGroup.getBlockSize()) { + if (idx * (long) cellSize >= blockGroup.getBlockSize()) { break; } StripingCell cell = new StripingCell(ecSchema, cellSize, idx); - startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize; + startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * (long) cellSize; if (startOffsets[cell.idxInStripe] < earliestStart) { earliestStart = startOffsets[cell.idxInStripe]; } From 3d734df24cba53ec56074b4d28e3bcdce7d2894e Mon Sep 17 00:00:00 2001 From: Walter Su Date: Fri, 22 May 2015 10:25:53 +0800 Subject: [PATCH 132/212] HDFS-8441. Erasure Coding: make condition check earlier for setReplication. (waltersu4549) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hdfs/server/namenode/FSDirAttrOp.java | 7 +++++- .../hdfs/server/namenode/FSNamesystem.java | 20 +++++++++++++++- .../hdfs/server/namenode/INodeFile.java | 2 -- .../hadoop/hdfs/TestErasureCodingZones.java | 24 ++++++++++++++++++- 5 files changed, 51 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index c986f190c46ae..d71b9c3064cf0 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -250,3 +250,6 @@ HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding. (Rakesh R via zhz) + + HDFS-8441. Erasure Coding: make condition check earlier for setReplication. + (waltersu4549) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index 9abb9fa82a4e3..d34a0fa694669 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -380,7 +380,7 @@ static INodeDirectory unprotectedSetQuota( static BlockInfoContiguous[] unprotectedSetReplication( FSDirectory fsd, String src, short replication, short[] blockRepls) throws QuotaExceededException, UnresolvedLinkException, - SnapshotAccessControlException { + SnapshotAccessControlException, UnsupportedActionException { assert fsd.hasWriteLock(); final INodesInPath iip = fsd.getINodesInPath4Write(src, true); @@ -389,6 +389,11 @@ static BlockInfoContiguous[] unprotectedSetReplication( return null; } INodeFile file = inode.asFile(); + if (file.isStriped()) { + throw new UnsupportedActionException( + "Cannot set replication to a file with striped blocks"); + } + final short oldBR = file.getPreferredBlockReplication(); // before setFileReplication, check for increasing block replication. 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 e9bb2f79f03af..a53cd61b20ec8 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 @@ -2404,7 +2404,18 @@ private HdfsFileStatus startFileInt(final String src, if (!DFSUtil.isValidName(src)) { throw new InvalidPathException(src); } - blockManager.verifyReplication(src, replication, clientMachine); + + checkOperation(OperationCategory.READ); + readLock(); + try { + checkOperation(OperationCategory.READ); + if (!isInECZone(src)) { + blockManager.verifyReplication(src, replication, clientMachine); + } + } finally { + readUnlock(); + } + checkOperation(OperationCategory.WRITE); if (blockSize < minBlockSize) { throw new IOException("Specified block size is less than configured" + @@ -7605,6 +7616,13 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, logAuditEvent(success, "createErasureCodingZone", srcArg, null, resultingStat); } + private boolean isInECZone(String src) throws IOException { + byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); + src = FSDirectory.resolvePath(src, pathComponents, dir); + final INodesInPath iip = dir.getINodesInPath(src, true); + return dir.isInECZone(iip); + } + /** * Get the erasure coding information for specified src */ 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 41287e87cf7f0..46880011b66da 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 @@ -429,8 +429,6 @@ private void setFileReplication(short replication) { /** Set the replication factor of this file. */ public final INodeFile setFileReplication(short replication, int latestSnapshotId) throws QuotaExceededException { - Preconditions.checkState(!isStriped(), - "Cannot set replication to a file with striped blocks"); recordModification(latestSnapshotId); setFileReplication(replication); return this; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index a7c3cd4d45fbf..480791ee199e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; @@ -35,6 +36,7 @@ import static org.junit.Assert.*; public class TestErasureCodingZones { + private Configuration conf; private MiniDFSCluster cluster; private DistributedFileSystem fs; private static final int BLOCK_SIZE = 1024; @@ -42,7 +44,7 @@ public class TestErasureCodingZones { @Before public void setupCluster() throws IOException { - Configuration conf = new HdfsConfiguration(); + conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); cluster = new MiniDFSCluster.Builder(conf). numDataNodes(1).build(); @@ -149,6 +151,26 @@ public void testMoveValidity() throws IOException, InterruptedException { } } + @Test + public void testReplication() throws IOException { + final Path testDir = new Path("/ec"); + fs.mkdir(testDir, FsPermission.getDirDefault()); + fs.createErasureCodingZone(testDir, null, 0); + final Path fooFile = new Path(testDir, "foo"); + // create ec file with replication=0 + fs.create(fooFile, FsPermission.getFileDefault(), true, + conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), + (short)0, fs.getDefaultBlockSize(fooFile), null); + + try { + fs.setReplication(fooFile, (short) 3); + fail("Shouldn't allow to set replication to a file with striped blocks"); + } catch (IOException e) { + assertExceptionContains( + "Cannot set replication to a file with striped blocks", e); + } + } + @Test public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { String src = "/ec"; From 4ae32abdf474f419d65464580f30c678ffff8182 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Fri, 22 May 2015 23:47:52 +0800 Subject: [PATCH 133/212] HADOOP-12013 Generate fixed data to perform erasure coder test. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/io/erasurecode/TestCoderBase.java | 87 ++++++++++++++++++- .../erasurecode/rawcoder/TestRSRawCoder.java | 8 +- .../rawcoder/TestRSRawCoderBase.java | 21 +++-- 4 files changed, 110 insertions(+), 8 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index c799b4fed3349..531b8d56ae24d 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -54,3 +54,5 @@ HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure coder. (Kai Zheng via Zhe Zhang) + + HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index cc3617cc436d9..3686695fb6d3c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -52,6 +52,12 @@ public abstract class TestCoderBase { // may go to different coding implementations. protected boolean usingDirectBuffer = true; + protected boolean usingFixedData = true; + // Using this the generated data can be repeatable across multiple calls to + // encode(), in order for troubleshooting. + private static int FIXED_DATA_GENERATOR = 0; + protected byte[][] fixedData; + protected int getChunkSize() { return chunkSize; } @@ -63,13 +69,17 @@ protected void setChunkSize(int chunkSize) { /** * Prepare before running the case. + * @param conf * @param numDataUnits * @param numParityUnits * @param erasedDataIndexes + * @param erasedParityIndexes + * @param usingFixedData Using fixed or pre-generated data to test instead of + * generating data */ protected void prepare(Configuration conf, int numDataUnits, int numParityUnits, int[] erasedDataIndexes, - int[] erasedParityIndexes) { + int[] erasedParityIndexes, boolean usingFixedData) { this.conf = conf; this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; @@ -77,6 +87,38 @@ protected void prepare(Configuration conf, int numDataUnits, erasedDataIndexes : new int[] {0}; this.erasedParityIndexes = erasedParityIndexes != null ? erasedParityIndexes : new int[] {0}; + this.usingFixedData = usingFixedData; + if (usingFixedData) { + prepareFixedData(); + } + } + + /** + * Prepare before running the case. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @param erasedDataIndexes + * @param erasedParityIndexes + */ + protected void prepare(Configuration conf, int numDataUnits, + int numParityUnits, int[] erasedDataIndexes, + int[] erasedParityIndexes) { + prepare(conf, numDataUnits, numParityUnits, erasedDataIndexes, + erasedParityIndexes, false); + } + + /** + * Prepare before running the case. + * @param numDataUnits + * @param numParityUnits + * @param erasedDataIndexes + * @param erasedParityIndexes + */ + protected void prepare(int numDataUnits, int numParityUnits, + int[] erasedDataIndexes, int[] erasedParityIndexes) { + prepare(null, numDataUnits, numParityUnits, erasedDataIndexes, + erasedParityIndexes, false); } /** @@ -278,6 +320,29 @@ protected ByteBuffer allocateOutputBuffer(int bufferLen) { * @return */ protected ECChunk[] prepareDataChunksForEncoding() { + if (usingFixedData) { + ECChunk[] chunks = new ECChunk[numDataUnits]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = makeChunkUsingData(fixedData[i]); + } + return chunks; + } + + return generateDataChunks(); + } + + private ECChunk makeChunkUsingData(byte[] data) { + ECChunk chunk = allocateOutputChunk(); + ByteBuffer buffer = chunk.getBuffer(); + int pos = buffer.position(); + buffer.put(data, 0, chunkSize); + buffer.flip(); + buffer.position(pos); + + return chunk; + } + + private ECChunk[] generateDataChunks() { ECChunk[] chunks = new ECChunk[numDataUnits]; for (int i = 0; i < chunks.length; i++) { chunks[i] = generateDataChunk(); @@ -286,6 +351,15 @@ protected ECChunk[] prepareDataChunksForEncoding() { return chunks; } + private void prepareFixedData() { + // We may load test data from a resource, or just generate randomly. + // The generated data will be used across subsequent encode/decode calls. + this.fixedData = new byte[numDataUnits][]; + for (int i = 0; i < numDataUnits; i++) { + fixedData[i] = generateFixedData(baseChunkSize * 2); + } + } + /** * Generate data chunk by making random data. * @return @@ -319,6 +393,17 @@ protected byte[] generateData(int len) { return buffer; } + protected byte[] generateFixedData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) FIXED_DATA_GENERATOR++; + if (FIXED_DATA_GENERATOR == 256) { + FIXED_DATA_GENERATOR = 0; + } + } + return buffer; + } + /** * Prepare parity chunks for encoding, each chunk for each parity unit. * @return diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 02b9eead3cc29..80ec04d0d4c4c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -80,7 +80,13 @@ public void testCodingBothBuffers_10x4_erasing_d0_p0() { @Test public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() { - prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}); + prepare(null, 10, 4, new int[]{2, 4}, new int[]{0}); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_usingFixedData_10x4_erasure_of_d2_d4_p0() { + prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}, true); testCoding(true); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java index c06aded7b976d..efde33211a8e6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java @@ -17,12 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; -import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; -import org.junit.Before; -import org.junit.Test; - -import java.nio.ByteBuffer; /** * Test base for raw Reed-solomon coders. @@ -32,6 +27,8 @@ public abstract class TestRSRawCoderBase extends TestRawCoderBase { private static int symbolSize = 0; private static int symbolMax = 0; + private static int RS_FIXED_DATA_GENERATOR = 0; + static { symbolSize = (int) Math.round(Math.log( RSUtil.GF.getFieldSize()) / Math.log(2)); @@ -41,9 +38,21 @@ public abstract class TestRSRawCoderBase extends TestRawCoderBase { @Override protected byte[] generateData(int len) { byte[] buffer = new byte[len]; - for (int i = 0; i < len; i++) { + for (int i = 0; i < buffer.length; i++) { buffer[i] = (byte) RAND.nextInt(symbolMax); } return buffer; } + + @Override + protected byte[] generateFixedData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) RS_FIXED_DATA_GENERATOR++; + if (RS_FIXED_DATA_GENERATOR == symbolMax) { + RS_FIXED_DATA_GENERATOR = 0; + } + } + return buffer; + } } From 0ed92e5b13f6bbc0ea7475bc67488359413a980e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 24 May 2015 23:07:34 +0800 Subject: [PATCH 134/212] HDFS-7768. Change fsck to support EC files. Contributed by Takanobu Asanuma --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../hdfs/server/namenode/NamenodeFsck.java | 206 +++++++++++++++--- .../hadoop/hdfs/server/namenode/TestFsck.java | 75 ++++++- 3 files changed, 238 insertions(+), 45 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index d71b9c3064cf0..efbadafdeb0a3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -253,3 +253,5 @@ HDFS-8441. Erasure Coding: make condition check earlier for setReplication. (waltersu4549) + + HDFS-7768. Change fsck to support EC files. (Takanobu Asanuma via szetszwo) 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 10673943dd96c..1c60a884dcaa3 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 @@ -74,6 +74,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.NodeBase; @@ -123,6 +124,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { private final int totalDatanodes; private final InetAddress remoteAddress; + private long totalDirs = 0L; + private long totalSymlinks = 0L; + private String lostFound = null; private boolean lfInited = false; private boolean lfInitedOk = false; @@ -356,13 +360,21 @@ public void fsck() { namenode.getNamesystem().getBlockManager().getStoragePolicies()); } - Result res = new Result(conf); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); - check(path, file, res); + check(path, file, replRes, ecRes); - out.println(res); - out.println(" Number of data-nodes:\t\t" + totalDatanodes); + out.print("\nStatus: "); + out.println(replRes.isHealthy() && ecRes.isHealthy() ? "HEALTHY" : "CORRUPT"); + out.println(" Number of data-nodes:\t" + totalDatanodes); out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks()); + out.println(" Total dirs:\t\t\t" + totalDirs); + out.println(" Total symlinks:\t\t" + totalSymlinks); + out.println("\nReplicated Blocks:"); + out.println(replRes); + out.println("\nErasure Coded Block Groups:"); + out.println(ecRes); if (this.showStoragePolcies) { out.print(storageTypeSummary.toString()); @@ -382,7 +394,7 @@ public void fsck() { // of file system and return appropriate code. Changing the output // string might break testcases. Also note this must be the last line // of the report. - if (res.isHealthy()) { + if (replRes.isHealthy() && ecRes.isHealthy()) { out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS); } else { out.print("\n\nThe filesystem under path '" + path + "' " + CORRUPT_STATUS); @@ -425,42 +437,49 @@ private void listCorruptFileBlocks() throws IOException { } @VisibleForTesting - void check(String parent, HdfsFileStatus file, Result res) throws IOException { + void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes) + throws IOException { String path = file.getFullName(parent); if (file.isDir()) { - checkDir(path, res); + checkDir(path, replRes, ecRes); return; } if (file.isSymlink()) { if (showFiles) { out.println(path + " "); } - res.totalSymlinks++; + totalSymlinks++; return; } LocatedBlocks blocks = getBlockLocations(path, file); if (blocks == null) { // the file is deleted return; } - collectFileSummary(path, file, res, blocks); - collectBlocksSummary(parent, file, res, blocks); + + final Result r = file.getReplication() == 0? ecRes: replRes; + collectFileSummary(path, file, r, blocks); + if (showprogress && (replRes.totalFiles + ecRes.totalFiles) % 100 == 0) { + out.println(); + out.flush(); + } + collectBlocksSummary(parent, file, r, blocks); } - private void checkDir(String path, Result res) throws IOException { + private void checkDir(String path, Result replRes, Result ecRes) throws IOException { if (snapshottableDirs != null && snapshottableDirs.contains(path)) { String snapshotPath = (path.endsWith(Path.SEPARATOR) ? path : path + Path.SEPARATOR) + HdfsConstants.DOT_SNAPSHOT_DIR; HdfsFileStatus snapshotFileInfo = namenode.getRpcServer().getFileInfo( snapshotPath); - check(snapshotPath, snapshotFileInfo, res); + check(snapshotPath, snapshotFileInfo, replRes, ecRes); } byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME; DirectoryListing thisListing; if (showFiles) { out.println(path + "

"); } - res.totalDirs++; + totalDirs++; do { assert lastReturnedName != null; thisListing = namenode.getRpcServer().getListing( @@ -470,7 +489,7 @@ private void checkDir(String path, Result res) throws IOException { } HdfsFileStatus[] files = thisListing.getPartialListing(); for (int i = 0; i < files.length; i++) { - check(path, files[i], res); + check(path, files[i], replRes, ecRes); } lastReturnedName = thisListing.getLastName(); } while (thisListing.hasMore()); @@ -517,10 +536,6 @@ private void collectFileSummary(String path, HdfsFileStatus file, Result res, } else if (showprogress) { out.print('.'); } - if ((showprogress) && res.totalFiles % 100 == 0) { - out.println(); - out.flush(); - } } private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res, @@ -543,7 +558,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res block.getLocalBlock()); // count decommissionedReplicas / decommissioningReplicas NumberReplicas numberReplicas = bm.countNodes(storedBlock); - int decommissionedReplicas = numberReplicas.decommissioned();; + int decommissionedReplicas = numberReplicas.decommissioned(); int decommissioningReplicas = numberReplicas.decommissioning(); res.decommissionedReplicas += decommissionedReplicas; res.decommissioningReplicas += decommissioningReplicas; @@ -555,7 +570,15 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res res.totalReplicas += totalReplicasPerBlock; // count expected replicas - short targetFileReplication = file.getReplication(); + short targetFileReplication; + if(file.getReplication() == 0) { + INode inode = namenode.getNamesystem().getFSDirectory().getINode(path); + INodesInPath iip = INodesInPath.fromINode(inode); + ECSchema ecSchema = namenode.getNamesystem().getFSDirectory().getECSchema(iip); + targetFileReplication = (short) (ecSchema.getNumDataUnits() + ecSchema.getNumParityUnits()); + } else { + targetFileReplication = file.getReplication(); + } res.numExpectedReplicas += targetFileReplication; // count under min repl'd blocks @@ -981,7 +1004,7 @@ static class Result { long missingReplicas = 0L; long decommissionedReplicas = 0L; long decommissioningReplicas = 0L; - long numUnderMinReplicatedBlocks=0L; + long numUnderMinReplicatedBlocks = 0L; long numOverReplicatedBlocks = 0L; long numUnderReplicatedBlocks = 0L; long numMisReplicatedBlocks = 0L; // blocks that do not satisfy block placement policy @@ -991,20 +1014,14 @@ static class Result { long totalOpenFilesBlocks = 0L; long totalFiles = 0L; long totalOpenFiles = 0L; - long totalDirs = 0L; - long totalSymlinks = 0L; long totalSize = 0L; long totalOpenFilesSize = 0L; long totalReplicas = 0L; - final short replication; final int minReplication; - Result(Configuration conf) { - this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, - DFSConfigKeys.DFS_REPLICATION_DEFAULT); - this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, - DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT); + Result(int minReplication) { + this.minReplication = minReplication; } /** @@ -1032,19 +1049,28 @@ float getReplicationFactor() { return 0.0f; return (float) (totalReplicas) / (float) totalBlocks; } + } + + @VisibleForTesting + static class ReplicationResult extends Result { + final short replication; + + ReplicationResult(Configuration conf) { + super(conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT)); + this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, + DFSConfigKeys.DFS_REPLICATION_DEFAULT); + } @Override public String toString() { StringBuilder res = new StringBuilder(); - res.append("Status: ").append((isHealthy() ? "HEALTHY" : "CORRUPT")) - .append("\n Total size:\t").append(totalSize).append(" B"); + res.append(" Total size:\t").append(totalSize).append(" B"); if (totalOpenFilesSize != 0) { res.append(" (Total open files size: ").append(totalOpenFilesSize) .append(" B)"); } - res.append("\n Total dirs:\t").append(totalDirs).append( - "\n Total files:\t").append(totalFiles); - res.append("\n Total symlinks:\t\t").append(totalSymlinks); + res.append("\n Total files:\t").append(totalFiles); if (totalOpenFiles != 0) { res.append(" (Files currently being written: ").append(totalOpenFiles) .append(")"); @@ -1134,4 +1160,116 @@ public String toString() { return res.toString(); } } + + @VisibleForTesting + static class ErasureCodingResult extends Result { + final String ecSchema; + + ErasureCodingResult(Configuration conf) { + this(ErasureCodingSchemaManager.getSystemDefaultSchema()); + } + + ErasureCodingResult(ECSchema ecSchema) { + super(ecSchema.getNumDataUnits()); + this.ecSchema = ecSchema.getSchemaName(); + } + + @Override + public String toString() { + StringBuilder res = new StringBuilder(); + res.append(" Total size:\t").append(totalSize).append(" B"); + if (totalOpenFilesSize != 0) { + res.append(" (Total open files size: ").append(totalOpenFilesSize) + .append(" B)"); + } + res.append("\n Total files:\t").append(totalFiles); + if (totalOpenFiles != 0) { + res.append(" (Files currently being written: ").append(totalOpenFiles) + .append(")"); + } + res.append("\n Total block groups (validated):\t").append(totalBlocks); + if (totalBlocks > 0) { + res.append(" (avg. block group size ").append((totalSize / totalBlocks)) + .append(" B)"); + } + if (totalOpenFilesBlocks != 0) { + res.append(" (Total open file block groups (not validated): ").append( + totalOpenFilesBlocks).append(")"); + } + if (corruptFiles > 0 || numUnderMinReplicatedBlocks > 0) { + res.append("\n ********************************"); + if(numUnderMinReplicatedBlocks>0){ + res.append("\n UNRECOVERABLE BLOCK GROUPS:\t").append(numUnderMinReplicatedBlocks); + if(totalBlocks>0){ + res.append(" (").append( + ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n ").append("MIN REQUIRED EC BLOCK:\t") + .append(minReplication); + } + if(corruptFiles>0) { + res.append( + "\n CORRUPT FILES:\t").append(corruptFiles); + if (missingSize > 0) { + res.append("\n MISSING BLOCK GROUPS:\t").append(missingIds.size()).append( + "\n MISSING SIZE:\t\t").append(missingSize).append(" B"); + } + if (corruptBlocks > 0) { + res.append("\n CORRUPT BLOCK GROUPS: \t").append(corruptBlocks).append( + "\n CORRUPT SIZE:\t\t").append(corruptSize).append(" B"); + } + } + res.append("\n ********************************"); + } + res.append("\n Minimally erasure-coded block groups:\t").append( + numMinReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numMinReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Over-erasure-coded block groups:\t") + .append(numOverReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numOverReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Under-erasure-coded block groups:\t").append( + numUnderReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numUnderReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Unsatisfactory placement block groups:\t\t") + .append(numMisReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Default schema:\t").append(ecSchema) + .append("\n Average block group size:\t").append( + getReplicationFactor()).append("\n Missing block groups:\t\t").append( + missingIds.size()).append("\n Corrupt block groups:\t\t").append( + corruptBlocks).append("\n Missing ec-blocks:\t\t").append( + missingReplicas); + if (totalReplicas > 0) { + res.append(" (").append( + ((float) (missingReplicas * 100) / (float) numExpectedReplicas)).append( + " %)"); + } + if (decommissionedReplicas > 0) { + res.append("\n Decommissioned ec-blocks:\t").append( + decommissionedReplicas); + } + if (decommissioningReplicas > 0) { + res.append("\n Decommissioning ec-blocks:\t").append( + decommissioningReplicas); + } + return res.toString(); + } + } } 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 b36beb2a5887b..e5482d414527d 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 @@ -84,6 +84,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result; +import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult; +import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.tools.DFSck; import org.apache.hadoop.io.IOUtils; @@ -1058,13 +1060,14 @@ public void testFsckMissingReplicas() throws IOException { final HdfsFileStatus file = namenode.getRpcServer().getFileInfo(pathString); assertNotNull(file); - Result res = new Result(conf); - fsck.check(pathString, file, res); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); + fsck.check(pathString, file, replRes, ecRes); // Also print the output from the fsck, for ex post facto sanity checks System.out.println(result.toString()); - assertEquals(res.missingReplicas, + assertEquals(replRes.missingReplicas, (NUM_BLOCKS*REPL_FACTOR) - (NUM_BLOCKS*NUM_REPLICAS)); - assertEquals(res.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR); + assertEquals(replRes.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR); } finally { if(dfs != null) { dfs.close(); @@ -1135,10 +1138,11 @@ public void testFsckMisPlacedReplicas() throws IOException { final HdfsFileStatus file = namenode.getRpcServer().getFileInfo(pathString); assertNotNull(file); - Result res = new Result(conf); - fsck.check(pathString, file, res); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); + fsck.check(pathString, file, replRes, ecRes); // check misReplicatedBlock number. - assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS); + assertEquals(replRes.numMisReplicatedBlocks, NUM_BLOCKS); } finally { if(dfs != null) { dfs.close(); @@ -1199,14 +1203,15 @@ public void testFsckFileNotFound() throws Exception { HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication, blockSize, modTime, accessTime, perms, owner, group, symlink, path, fileId, numChildren, null, storagePolicy, null, 0); - Result res = new Result(conf); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); try { - fsck.check(pathString, file, res); + fsck.check(pathString, file, replRes, ecRes); } catch (Exception e) { fail("Unexpected exception "+ e.getMessage()); } - assertTrue(res.toString().contains("HEALTHY")); + assertTrue(replRes.isHealthy()); } /** Test fsck with symlinks in the filesystem */ @@ -1629,4 +1634,52 @@ public void testFsckWithDecommissionedReplicas() throws Exception { } } } -} + + @Test + public void testECFsck() throws Exception { + MiniDFSCluster cluster = null; + FileSystem fs = null; + try { + Configuration conf = new HdfsConfiguration(); + final long precision = 1L; + conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision); + conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L); + int totalSize = ErasureCodingSchemaManager.getSystemDefaultSchema().getNumDataUnits() + + ErasureCodingSchemaManager.getSystemDefaultSchema().getNumParityUnits(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build(); + fs = cluster.getFileSystem(); + Path replDirPath = new Path("/replicated"); + Path replFilePath = new Path(replDirPath, "replfile"); + final short factor = 3; + DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0); + DFSTestUtil.waitReplication(fs, replFilePath, factor); + Path ecDirPath = new Path("/striped"); + Path ecFilePath = new Path(ecDirPath, "ecfile"); + final int numBlocks = 4; + DFSTestUtil.createStripedFile(cluster, ecFilePath, ecDirPath, numBlocks, 2, true); + long replTime = fs.getFileStatus(replFilePath).getAccessTime(); + long ecTime = fs.getFileStatus(ecFilePath).getAccessTime(); + Thread.sleep(precision); + setupAuditLogs(); + String outStr = runFsck(conf, 0, true, "/"); + verifyAuditLogs(); + assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime()); + assertEquals(ecTime, fs.getFileStatus(ecFilePath).getAccessTime()); + System.out.println(outStr); + assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS)); + if (fs != null) {try{fs.close();} catch(Exception e){}} + cluster.shutdown(); + + // restart the cluster; bring up namenode but not the data nodes + cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(0).format(false).build(); + outStr = runFsck(conf, 1, true, "/"); + // expect the result is corrupt + assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS)); + System.out.println(outStr); + } finally { + if (fs != null) {try{fs.close();} catch(Exception e){}} + if (cluster != null) { cluster.shutdown(); } + } + } +} \ No newline at end of file From b30e96bfb4b8ce5537671c97f0c9c56cd195bfdc Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Mon, 25 May 2015 16:13:29 +0800 Subject: [PATCH 135/212] HDFS-8382 Remove chunkSize and initialize from erasure coder. Contributed by Kai Zheng --- .../codec/AbstractErasureCodec.java | 43 ++----------------- .../io/erasurecode/codec/ErasureCodec.java | 7 --- .../io/erasurecode/codec/RSErasureCodec.java | 13 ++++-- .../io/erasurecode/codec/XORErasureCodec.java | 13 +++--- .../coder/AbstractErasureCoder.java | 39 +++++++---------- .../coder/AbstractErasureDecoder.java | 9 ++++ .../coder/AbstractErasureEncoder.java | 9 ++++ .../io/erasurecode/coder/ErasureCoder.java | 27 ++---------- .../erasurecode/coder/RSErasureDecoder.java | 16 +++++-- .../erasurecode/coder/RSErasureEncoder.java | 16 +++++-- .../erasurecode/coder/XORErasureDecoder.java | 14 ++++-- .../erasurecode/coder/XORErasureEncoder.java | 16 +++++-- .../rawcoder/AbstractRawErasureCoder.java | 15 ++----- .../rawcoder/AbstractRawErasureDecoder.java | 4 ++ .../rawcoder/AbstractRawErasureEncoder.java | 4 ++ .../io/erasurecode/rawcoder/RSRawDecoder.java | 5 +-- .../io/erasurecode/rawcoder/RSRawEncoder.java | 6 +-- .../rawcoder/RSRawErasureCoderFactory.java | 8 ++-- .../erasurecode/rawcoder/RawErasureCoder.java | 14 ------ .../rawcoder/RawErasureCoderFactory.java | 8 +++- .../erasurecode/rawcoder/XORRawDecoder.java | 4 ++ .../erasurecode/rawcoder/XORRawEncoder.java | 4 ++ .../rawcoder/XORRawErasureCoderFactory.java | 8 ++-- .../coder/TestErasureCoderBase.java | 42 ++++++++++-------- .../rawcoder/TestRawCoderBase.java | 15 ++++--- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/hdfs/DFSStripedOutputStream.java | 3 +- .../erasurecode/ErasureCodingWorker.java | 14 +++--- .../hadoop/hdfs/util/StripedBlockUtil.java | 3 +- .../hdfs/TestDFSStripedOutputStream.java | 4 +- .../hadoop/hdfs/TestWriteReadStripedFile.java | 3 +- 31 files changed, 187 insertions(+), 201 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java index 999378689384b..0cacfbcacd626 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java @@ -19,7 +19,6 @@ import org.apache.hadoop.conf.Configured; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.coder.*; import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper; /** @@ -28,10 +27,9 @@ public abstract class AbstractErasureCodec extends Configured implements ErasureCodec { - private ECSchema schema; + private final ECSchema schema; - @Override - public void setSchema(ECSchema schema) { + public AbstractErasureCodec(ECSchema schema) { this.schema = schema; } @@ -39,7 +37,7 @@ public String getName() { return schema.getCodecName(); } - protected ECSchema getSchema() { + public ECSchema getSchema() { return schema; } @@ -50,39 +48,4 @@ public BlockGrouper createBlockGrouper() { return blockGrouper; } - - @Override - public ErasureCoder createEncoder() { - ErasureCoder encoder = doCreateEncoder(); - prepareErasureCoder(encoder); - return encoder; - } - - /** - * Create a new encoder instance to be initialized afterwards. - * @return encoder - */ - protected abstract ErasureCoder doCreateEncoder(); - - @Override - public ErasureCoder createDecoder() { - ErasureCoder decoder = doCreateDecoder(); - prepareErasureCoder(decoder); - return decoder; - } - - /** - * Create a new decoder instance to be initialized afterwards. - * @return decoder - */ - protected abstract ErasureCoder doCreateDecoder(); - - private void prepareErasureCoder(ErasureCoder erasureCoder) { - if (getSchema() == null) { - throw new RuntimeException("No schema been set yet"); - } - - erasureCoder.setConf(getConf()); - erasureCoder.initialize(getSchema()); - } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java index e639484ee2b17..9aa3db265e30f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java @@ -18,7 +18,6 @@ package org.apache.hadoop.io.erasurecode.codec; import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.coder.ErasureCoder; import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper; @@ -29,12 +28,6 @@ */ public interface ErasureCodec extends Configurable { - /** - * Set EC schema to be used by this codec. - * @param schema - */ - public void setSchema(ECSchema schema); - /** * Create block grouper * @return block grouper diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java index 9e91b6019ce61..6edd638dc5406 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.codec; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.coder.ErasureCoder; import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder; import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder; @@ -26,13 +27,17 @@ */ public class RSErasureCodec extends AbstractErasureCodec { + public RSErasureCodec(ECSchema schema) { + super(schema); + } + @Override - protected ErasureCoder doCreateEncoder() { - return new RSErasureEncoder(); + public ErasureCoder createEncoder() { + return new RSErasureEncoder(getSchema()); } @Override - protected ErasureCoder doCreateDecoder() { - return new RSErasureDecoder(); + public ErasureCoder createDecoder() { + return new RSErasureDecoder(getSchema()); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java index 0f726d7cd0bf7..e2dcfa7983c8a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java @@ -27,19 +27,18 @@ */ public class XORErasureCodec extends AbstractErasureCodec { - @Override - public void setSchema(ECSchema schema) { - super.setSchema(schema); + public XORErasureCodec(ECSchema schema) { + super(schema); assert(schema.getNumParityUnits() == 1); } @Override - protected ErasureCoder doCreateEncoder() { - return new XORErasureEncoder(); + public ErasureCoder createEncoder() { + return new XORErasureEncoder(getSchema()); } @Override - protected ErasureCoder doCreateDecoder() { - return new XORErasureDecoder(); + public ErasureCoder createDecoder() { + return new XORErasureDecoder(getSchema()); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index d491570f61f86..c572badef2e12 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -33,18 +33,18 @@ public abstract class AbstractErasureCoder extends Configured implements ErasureCoder { - private int numDataUnits; - private int numParityUnits; - private int chunkSize; + private final int numDataUnits; + private final int numParityUnits; /** * Create raw decoder using the factory specified by rawCoderFactoryKey * @param rawCoderFactoryKey * @return raw decoder */ - protected RawErasureDecoder createRawDecoder(String rawCoderFactoryKey) { + protected RawErasureDecoder createRawDecoder( + String rawCoderFactoryKey, int dataUnitsCount, int parityUnitsCount) { RawErasureCoder rawCoder = createRawCoder(getConf(), - rawCoderFactoryKey, false); + rawCoderFactoryKey, false, dataUnitsCount, parityUnitsCount); return (RawErasureDecoder) rawCoder; } @@ -53,9 +53,10 @@ protected RawErasureDecoder createRawDecoder(String rawCoderFactoryKey) { * @param rawCoderFactoryKey * @return raw encoder */ - protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) { + protected RawErasureEncoder createRawEncoder( + String rawCoderFactoryKey, int dataUnitsCount, int parityUnitsCount) { RawErasureCoder rawCoder = createRawCoder(getConf(), - rawCoderFactoryKey, true); + rawCoderFactoryKey, true, dataUnitsCount, parityUnitsCount); return (RawErasureEncoder) rawCoder; } @@ -67,7 +68,8 @@ protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) { * @return raw coder */ public static RawErasureCoder createRawCoder(Configuration conf, - String rawCoderFactoryKey, boolean isEncoder) { + String rawCoderFactoryKey, boolean isEncoder, int numDataUnits, + int numParityUnits) { if (conf == null) { return null; @@ -90,21 +92,17 @@ public static RawErasureCoder createRawCoder(Configuration conf, throw new RuntimeException("Failed to create raw coder", e); } - return isEncoder ? fact.createEncoder() : fact.createDecoder(); + return isEncoder ? fact.createEncoder(numDataUnits, numParityUnits) : + fact.createDecoder(numDataUnits, numParityUnits); } - @Override - public void initialize(int numDataUnits, int numParityUnits, - int chunkSize) { + public AbstractErasureCoder(int numDataUnits, int numParityUnits) { this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; - this.chunkSize = chunkSize; } - @Override - public void initialize(ECSchema schema) { - initialize(schema.getNumDataUnits(), schema.getNumParityUnits(), - schema.getChunkSize()); + public AbstractErasureCoder(ECSchema schema) { + this(schema.getNumDataUnits(), schema.getNumParityUnits()); } @Override @@ -118,12 +116,7 @@ public int getNumParityUnits() { } @Override - public int getChunkSize() { - return chunkSize; - } - - @Override - public boolean preferNativeBuffer() { + public boolean preferDirectBuffer() { return false; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java index 6437236355dc4..3ea9311d194ad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java @@ -19,6 +19,7 @@ import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; /** * An abstract erasure decoder that's to be inherited by new decoders. @@ -27,6 +28,14 @@ */ public abstract class AbstractErasureDecoder extends AbstractErasureCoder { + public AbstractErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public AbstractErasureDecoder(ECSchema schema) { + super(schema); + } + @Override public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) { // We may have more than this when considering complicate cases. HADOOP-11550 diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java index a836b75e7a58d..7c887e8e2cc8d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java @@ -19,6 +19,7 @@ import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; /** * An abstract erasure encoder that's to be inherited by new encoders. @@ -27,6 +28,14 @@ */ public abstract class AbstractErasureEncoder extends AbstractErasureCoder { + public AbstractErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public AbstractErasureEncoder(ECSchema schema) { + super(schema); + } + @Override public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) { // We may have more than this when considering complicate cases. HADOOP-11550 diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java index 64a82eaf6ae8a..f05ea417463f6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java @@ -19,7 +19,6 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.io.erasurecode.ECBlockGroup; -import org.apache.hadoop.io.erasurecode.ECSchema; /** * An erasure coder to perform encoding or decoding given a group. Generally it @@ -37,20 +36,6 @@ */ public interface ErasureCoder extends Configurable { - /** - * Initialize with the important parameters for the code. - * @param numDataUnits how many data inputs for the coding - * @param numParityUnits how many parity outputs the coding generates - * @param chunkSize the size of the input/output buffer - */ - public void initialize(int numDataUnits, int numParityUnits, int chunkSize); - - /** - * Initialize with an EC schema. - * @param schema - */ - public void initialize(ECSchema schema); - /** * The number of data input units for the coding. A unit can be a byte, * chunk or buffer or even a block. @@ -65,12 +50,6 @@ public interface ErasureCoder extends Configurable { */ public int getNumParityUnits(); - /** - * Chunk buffer size for the input/output - * @return chunk buffer size - */ - public int getChunkSize(); - /** * Calculate the encoding or decoding steps given a block blockGroup. * @@ -83,13 +62,13 @@ public interface ErasureCoder extends Configurable { public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup); /** - * Tell if native or off-heap buffer is preferred or not. It's for callers to + * Tell if direct or off-heap buffer is preferred or not. It's for callers to * decide how to allocate coding chunk buffers, either on heap or off heap. * It will return false by default. - * @return true if native buffer is preferred for performance consideration, + * @return true if direct buffer is preferred for performance consideration, * otherwise false. */ - public boolean preferNativeBuffer(); + public boolean preferDirectBuffer(); /** * Release the resources if any. Good chance to invoke RawErasureCoder#release. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java index ec7cbb5de5276..57f4373ac1772 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; @@ -31,6 +32,14 @@ public class RSErasureDecoder extends AbstractErasureDecoder { private RawErasureDecoder rsRawDecoder; + public RSErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public RSErasureDecoder(ECSchema schema) { + super(schema); + } + @Override protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { @@ -45,12 +54,11 @@ protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { private RawErasureDecoder checkCreateRSRawDecoder() { if (rsRawDecoder == null) { rsRawDecoder = createRawDecoder( - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY); + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + getNumDataUnits(), getNumParityUnits()); if (rsRawDecoder == null) { - rsRawDecoder = new RSRawDecoder(); + rsRawDecoder = new RSRawDecoder(getNumDataUnits(), getNumParityUnits()); } - rsRawDecoder.initialize(getNumDataUnits(), - getNumParityUnits(), getChunkSize()); } return rsRawDecoder; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java index 18ca5acd910a4..ab23474f54cdf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; @@ -31,6 +32,14 @@ public class RSErasureEncoder extends AbstractErasureEncoder { private RawErasureEncoder rawEncoder; + public RSErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public RSErasureEncoder(ECSchema schema) { + super(schema); + } + @Override protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { @@ -45,12 +54,11 @@ protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { private RawErasureEncoder checkCreateRSRawEncoder() { if (rawEncoder == null) { rawEncoder = createRawEncoder( - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY); + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + getNumDataUnits(), getNumParityUnits()); if (rawEncoder == null) { - rawEncoder = new RSRawEncoder(); + rawEncoder = new RSRawEncoder(getNumDataUnits(), getNumParityUnits()); } - rawEncoder.initialize(getNumDataUnits(), - getNumParityUnits(), getChunkSize()); } return rawEncoder; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java index 067254912eee9..3fe8d1bf82d64 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java @@ -19,6 +19,7 @@ import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder; @@ -29,12 +30,19 @@ */ public class XORErasureDecoder extends AbstractErasureDecoder { + public XORErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public XORErasureDecoder(ECSchema schema) { + super(schema); + } + @Override protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { // May be configured - RawErasureDecoder rawDecoder = new XORRawDecoder(); - rawDecoder.initialize(getNumDataUnits(), - getNumParityUnits(), getChunkSize()); + RawErasureDecoder rawDecoder = new XORRawDecoder( + getNumDataUnits(), getNumParityUnits()); ECBlock[] inputBlocks = getInputBlocks(blockGroup); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java index 90118573893e9..5020896b66d46 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java @@ -19,22 +19,30 @@ import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder; /** * Xor erasure encoder that encodes a block group. * - * It implements {@link ErasureEncoder}. + * It implements {@link ErasureCoder}. */ public class XORErasureEncoder extends AbstractErasureEncoder { + public XORErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public XORErasureEncoder(ECSchema schema) { + super(schema); + } + @Override protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { // May be configured - RawErasureEncoder rawEncoder = new XORRawEncoder(); - rawEncoder.initialize(getNumDataUnits(), - getNumParityUnits(), getChunkSize()); + RawErasureEncoder rawEncoder = new XORRawEncoder( + getNumDataUnits(), getNumParityUnits()); ECBlock[] inputBlocks = getInputBlocks(blockGroup); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index 52689620a105f..06ae660f36728 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -31,16 +31,12 @@ public abstract class AbstractRawErasureCoder extends Configured implements RawErasureCoder { - private int numDataUnits; - private int numParityUnits; - private int chunkSize; + private final int numDataUnits; + private final int numParityUnits; - @Override - public void initialize(int numDataUnits, int numParityUnits, - int chunkSize) { + public AbstractRawErasureCoder(int numDataUnits, int numParityUnits) { this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; - this.chunkSize = chunkSize; } @Override @@ -53,11 +49,6 @@ public int getNumParityUnits() { return numParityUnits; } - @Override - public int getChunkSize() { - return chunkSize; - } - @Override public boolean preferDirectBuffer() { return false; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java index 31f4fb858cd27..0c1f80f968380 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -30,6 +30,10 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder implements RawErasureDecoder { + public AbstractRawErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + @Override public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java index 0ae54c5531651..c7a136b0f4f99 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -30,6 +30,10 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder implements RawErasureEncoder { + public AbstractRawErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + @Override public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { checkParameters(inputs, outputs); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java index ff1162f494340..e265dcef5d395 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java @@ -31,9 +31,8 @@ public class RSRawDecoder extends AbstractRawErasureDecoder { private int[] errSignature; private int[] primitivePower; - @Override - public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { - super.initialize(numDataUnits, numParityUnits, chunkSize); + public RSRawDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); this.errSignature = new int[numParityUnits]; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java index 9136331fbd6db..efeee9030c9b5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java @@ -29,9 +29,9 @@ public class RSRawEncoder extends AbstractRawErasureEncoder { private int[] generatingPolynomial; - @Override - public void initialize(int numDataUnits, int numParityUnits, int chunkSize) { - super.initialize(numDataUnits, numParityUnits, chunkSize); + public RSRawEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); int[] primitivePower = RSUtil.getPrimitivePower(numDataUnits, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java index 19a95af7ad4c8..5db49e388fb36 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java @@ -23,12 +23,12 @@ public class RSRawErasureCoderFactory implements RawErasureCoderFactory { @Override - public RawErasureEncoder createEncoder() { - return new RSRawEncoder(); + public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) { + return new RSRawEncoder(numDataUnits, numParityUnits); } @Override - public RawErasureDecoder createDecoder() { - return new RSRawDecoder(); + public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) { + return new RSRawDecoder(numDataUnits, numParityUnits); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java index 3fb211fd67808..0ddb460345cf7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -35,14 +35,6 @@ */ public interface RawErasureCoder extends Configurable { - /** - * Initialize with the important parameters for the code. - * @param numDataUnits how many data inputs for the coding - * @param numParityUnits how many parity outputs the coding generates - * @param chunkSize the size of the input/output buffer - */ - public void initialize(int numDataUnits, int numParityUnits, int chunkSize); - /** * The number of data input units for the coding. A unit can be a byte, * chunk or buffer or even a block. @@ -57,12 +49,6 @@ public interface RawErasureCoder extends Configurable { */ public int getNumParityUnits(); - /** - * Chunk buffer size for the input/output - * @return chunk buffer size - */ - public int getChunkSize(); - /** * Tell if direct buffer is preferred or not. It's for callers to * decide how to allocate coding chunk buffers, using DirectByteBuffer or diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java index 95a1cfe383a8d..26eddfc6681bc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java @@ -26,13 +26,17 @@ public interface RawErasureCoderFactory { /** * Create raw erasure encoder. + * @param numDataUnits + * @param numParityUnits * @return raw erasure encoder */ - public RawErasureEncoder createEncoder(); + public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits); /** * Create raw erasure decoder. + * @param numDataUnits + * @param numParityUnits * @return raw erasure decoder */ - public RawErasureDecoder createDecoder(); + public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java index bf6e8942708c6..a09105c4ad58c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java @@ -28,6 +28,10 @@ */ public class XORRawDecoder extends AbstractRawErasureDecoder { + public XORRawDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + @Override protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java index feffbbf015825..894f20c7607e8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java @@ -28,6 +28,10 @@ */ public class XORRawEncoder extends AbstractRawErasureEncoder { + public XORRawEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { ByteBuffer output = outputs[0]; resetOutputBuffer(output); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java index 67f45c161b5bd..de20c950eb795 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java @@ -23,12 +23,12 @@ public class XORRawErasureCoderFactory implements RawErasureCoderFactory { @Override - public RawErasureEncoder createEncoder() { - return new XORRawEncoder(); + public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) { + return new XORRawEncoder(numDataUnits, numParityUnits); } @Override - public RawErasureDecoder createDecoder() { - return new XORRawDecoder(); + public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) { + return new XORRawDecoder(numDataUnits, numParityUnits); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index 154ec1869944d..f9666b6b4b941 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -22,6 +22,8 @@ import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.TestCoderBase; +import java.lang.reflect.Constructor; + /** * Erasure coder test base with utilities. */ @@ -139,46 +141,50 @@ protected void compareAndVerify(ECBlock[] erasedBlocks, } } + private void prepareCoders() { + if (encoder == null) { + encoder = createEncoder(); + } + + if (decoder == null) { + decoder = createDecoder(); + } + } + /** - * Create erasure encoder for test. + * Create the raw erasure encoder to test * @return */ - private ErasureCoder createEncoder() { + protected ErasureCoder createEncoder() { ErasureCoder encoder; try { - encoder = encoderClass.newInstance(); + Constructor constructor = + (Constructor) + encoderClass.getConstructor(int.class, int.class); + encoder = constructor.newInstance(numDataUnits, numParityUnits); } catch (Exception e) { throw new RuntimeException("Failed to create encoder", e); } - encoder.initialize(numDataUnits, numParityUnits, getChunkSize()); encoder.setConf(getConf()); return encoder; } - private void prepareCoders() { - if (encoder == null) { - encoder = createEncoder(); - } - - if (decoder == null) { - decoder = createDecoder(); - } - } - /** - * Create the erasure decoder for the test. + * create the raw erasure decoder to test * @return */ - private ErasureCoder createDecoder() { + protected ErasureCoder createDecoder() { ErasureCoder decoder; try { - decoder = decoderClass.newInstance(); + Constructor constructor = + (Constructor) + decoderClass.getConstructor(int.class, int.class); + decoder = constructor.newInstance(numDataUnits, numParityUnits); } catch (Exception e) { throw new RuntimeException("Failed to create decoder", e); } - decoder.initialize(numDataUnits, numParityUnits, getChunkSize()); decoder.setConf(getConf()); return decoder; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 45823175cf12e..cfaa2c542a523 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -21,6 +21,8 @@ import org.apache.hadoop.io.erasurecode.TestCoderBase; import org.junit.Assert; +import java.lang.reflect.Constructor; + /** * Raw coder test base with utilities. */ @@ -136,12 +138,14 @@ private void prepareCoders() { protected RawErasureEncoder createEncoder() { RawErasureEncoder encoder; try { - encoder = encoderClass.newInstance(); + Constructor constructor = + (Constructor) + encoderClass.getConstructor(int.class, int.class); + encoder = constructor.newInstance(numDataUnits, numParityUnits); } catch (Exception e) { throw new RuntimeException("Failed to create encoder", e); } - encoder.initialize(numDataUnits, numParityUnits, getChunkSize()); encoder.setConf(getConf()); return encoder; } @@ -153,14 +157,15 @@ protected RawErasureEncoder createEncoder() { protected RawErasureDecoder createDecoder() { RawErasureDecoder decoder; try { - decoder = decoderClass.newInstance(); + Constructor constructor = + (Constructor) + decoderClass.getConstructor(int.class, int.class); + decoder = constructor.newInstance(numDataUnits, numParityUnits); } catch (Exception e) { throw new RuntimeException("Failed to create decoder", e); } - decoder.initialize(numDataUnits, numParityUnits, getChunkSize()); decoder.setConf(getConf()); return decoder; } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index efbadafdeb0a3..4609fb6a8b9b2 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -255,3 +255,5 @@ (waltersu4549) HDFS-7768. Change fsck to support EC files. (Takanobu Asanuma via szetszwo) + + HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 515ce0cdf7b2f..0935d5c634a46 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -247,8 +247,7 @@ ExtendedBlock getBlock() { numDataBlocks = schema.getNumDataUnits(); numAllBlocks = numDataBlocks + numParityBlocks; - encoder = new RSRawEncoder(); - encoder.initialize(numDataBlocks, numParityBlocks, cellSize); + encoder = new RSRawEncoder(numDataBlocks, numParityBlocks); coordinator = new Coordinator(dfsClient.getConf(), numDataBlocks, numAllBlocks); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index ded51eb585634..00cf0fd5e2c3a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -111,12 +111,12 @@ public ErasureCodingWorker(Configuration conf, DataNode datanode) { DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT); } - private RawErasureEncoder newEncoder() { - return new RSRawEncoder(); + private RawErasureEncoder newEncoder(int numDataUnits, int numParityUnits) { + return new RSRawEncoder(numDataUnits, numParityUnits); } - private RawErasureDecoder newDecoder() { - return new RSRawDecoder(); + private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) { + return new RSRawDecoder(numDataUnits, numParityUnits); } private void initializeStripedReadThreadPool(int num) { @@ -517,16 +517,14 @@ private void paddingBufferToLen(ByteBuffer buffer, int len) { // Initialize encoder private void initEncoderIfNecessary() { if (encoder == null) { - encoder = newEncoder(); - encoder.initialize(dataBlkNum, parityBlkNum, bufferSize); + encoder = newEncoder(dataBlkNum, parityBlkNum); } } // Initialize decoder private void initDecoderIfNecessary() { if (decoder == null) { - decoder = newDecoder(); - decoder.initialize(dataBlkNum, parityBlkNum, bufferSize); + decoder = newDecoder(dataBlkNum, parityBlkNum); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 38dc61aa33572..8f63236aa44f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -287,8 +287,7 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf, } byte[][] outputs = new byte[parityBlkNum][(int) alignedStripe.getSpanInBlock()]; - RSRawDecoder rsRawDecoder = new RSRawDecoder(); - rsRawDecoder.initialize(dataBlkNum, parityBlkNum, (int) alignedStripe.getSpanInBlock()); + RSRawDecoder rsRawDecoder = new RSRawDecoder(dataBlkNum, parityBlkNum); rsRawDecoder.decode(decodeInputs, decodeIndices, outputs); for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index e795ea7a8edc1..e041dbe303e66 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -274,8 +274,8 @@ static void verifyParity(final long size, final int cellSize, System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length); } } - final RawErasureEncoder encoder = new RSRawEncoder(); - encoder.initialize(dataBytes.length, parityBytes.length, cellSize); + final RawErasureEncoder encoder = + new RSRawEncoder(dataBytes.length, parityBytes.length); encoder.encode(dataBytes, expectedParityBytes); for (int i = 0; i < parityBytes.length; i++) { if (i != killedDnIndex) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 1976dcaa86627..70802fb59a826 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -382,8 +382,7 @@ public void testWritePreadWithDNFailure() throws IOException { Assert.assertEquals("The length of file should be the same to write size", length - startOffsetInFile, readLen); - RSRawDecoder rsRawDecoder = new RSRawDecoder(); - rsRawDecoder.initialize(dataBlocks, parityBlocks, 1); + RSRawDecoder rsRawDecoder = new RSRawDecoder(dataBlocks, parityBlocks); byte[] expected = new byte[readLen]; for (int i = startOffsetInFile; i < length; i++) { //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 From 9a18598e2da8e699ed852ffa30fd7f503902190c Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 25 May 2015 11:57:17 +0530 Subject: [PATCH 136/212] HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../org/apache/hadoop/hdfs/DFSClient.java | 22 ++----- .../hadoop/hdfs/DistributedFileSystem.java | 16 ++--- .../hadoop/hdfs/protocol/ClientProtocol.java | 12 +--- .../hdfs/protocol/ErasureCodingInfo.java | 41 ------------ .../hdfs/protocol/ErasureCodingZoneInfo.java | 66 ------------------- ...amenodeProtocolServerSideTranslatorPB.java | 37 +++-------- .../ClientNamenodeProtocolTranslatorPB.java | 34 +++------- .../hadoop/hdfs/protocolPB/PBHelper.java | 34 ++++------ .../server/blockmanagement/BlockManager.java | 12 ++-- .../namenode/ErasureCodingZoneManager.java | 10 +-- .../namenode/FSDirStatAndListingOp.java | 14 ++-- .../hdfs/server/namenode/FSDirectory.java | 6 +- .../hdfs/server/namenode/FSNamesystem.java | 23 ++----- .../server/namenode/NameNodeRpcServer.java | 13 +--- .../hdfs/server/namenode/Namesystem.java | 10 ++- .../hdfs/tools/erasurecode/ECCommand.java | 16 ++--- .../main/proto/ClientNamenodeProtocol.proto | 6 +- .../src/main/proto/erasurecoding.proto | 26 ++------ .../hadoop/hdfs/TestErasureCodingZones.java | 16 ++--- .../test/resources/testErasureCodingConf.xml | 38 +++++------ 21 files changed, 117 insertions(+), 337 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 4609fb6a8b9b2..d045ee5fb2187 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -257,3 +257,5 @@ HDFS-7768. Change fsck to support EC files. (Takanobu Asanuma via szetszwo) HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng) + + HDFS-8408. Revisit and refactor ErasureCodingInfo (vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 42f48f0959666..a2a622157c859 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -119,8 +119,7 @@ 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.ErasureCodingInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -3137,19 +3136,6 @@ public void checkAccess(String src, FsAction mode) throws IOException { } } - public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException { - checkOpen(); - TraceScope scope = getPathTraceScope("getErasureCodingInfo", src); - try { - return namenode.getErasureCodingInfo(src); - } catch (RemoteException re) { - throw re.unwrapRemoteException(AccessControlException.class, - FileNotFoundException.class, UnresolvedPathException.class); - } finally { - scope.close(); - } - } - public ECSchema[] getECSchemas() throws IOException { checkOpen(); TraceScope scope = Trace.startSpan("getECSchemas", traceSampler); @@ -3359,11 +3345,11 @@ TraceScope getSrcDstTraceScope(String description, String src, String dst) { * @return Returns the zone information if path is in EC Zone, null otherwise * @throws IOException */ - public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + public ErasureCodingZone getErasureCodingZone(String src) throws IOException { checkOpen(); - TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src); + TraceScope scope = getPathTraceScope("getErasureCodingZone", src); try { - return namenode.getErasureCodingZoneInfo(src); + return namenode.getErasureCodingZone(src); } catch (RemoteException re) { throw re.unwrapRemoteException(FileNotFoundException.class, AccessControlException.class, UnresolvedPathException.class); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 2e21372593aa9..6d55922058294 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -75,7 +75,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -2316,25 +2316,25 @@ public Void next(final FileSystem fs, final Path p) throws IOException { * @return Returns the zone information if path is in EC zone, null otherwise * @throws IOException */ - public ErasureCodingZoneInfo getErasureCodingZoneInfo(final Path path) + public ErasureCodingZone getErasureCodingZone(final Path path) throws IOException { Path absF = fixRelativePart(path); - return new FileSystemLinkResolver() { + return new FileSystemLinkResolver() { @Override - public ErasureCodingZoneInfo doCall(final Path p) throws IOException, + public ErasureCodingZone doCall(final Path p) throws IOException, UnresolvedLinkException { - return dfs.getErasureCodingZoneInfo(getPathName(p)); + return dfs.getErasureCodingZone(getPathName(p)); } @Override - public ErasureCodingZoneInfo next(final FileSystem fs, final Path p) + public ErasureCodingZone next(final FileSystem fs, final Path p) throws IOException { if (fs instanceof DistributedFileSystem) { DistributedFileSystem myDfs = (DistributedFileSystem) fs; - return myDfs.getErasureCodingZoneInfo(p); + return myDfs.getErasureCodingZone(p); } throw new UnsupportedOperationException( - "Cannot getErasureCodingZoneInfo through a symlink to a " + "Cannot getErasureCodingZone through a symlink to a " + "non-DistributedFileSystem: " + path + " -> " + p); } }.resolve(this, absF); 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 4f985ba60736a..d2b8a517e752e 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 @@ -1466,16 +1466,6 @@ public List listXAttrs(String src) public void createErasureCodingZone(String src, ECSchema schema, int cellSize) throws IOException; - /** - * Gets the ECInfo for the specified file/directory - * - * @param src - * @return Returns the ECInfo if the file/directory is erasure coded, null otherwise - * @throws IOException - */ - @Idempotent - public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException; - /** * Gets list of ECSchemas loaded in Namenode * @@ -1492,5 +1482,5 @@ public void createErasureCodingZone(String src, ECSchema schema, int cellSize) * @throws IOException */ @Idempotent - public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException; + public ErasureCodingZone getErasureCodingZone(String src) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java deleted file mode 100644 index bad09b3d19012..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hdfs.protocol; - -import org.apache.hadoop.io.erasurecode.ECSchema; - -/** - * Class to provide information, such as ECSchema, for a file/block. - */ -public class ErasureCodingInfo { - private final String src; - private final ECSchema schema; - - public ErasureCodingInfo(String src, ECSchema schema) { - this.src = src; - this.schema = schema; - } - - public String getSrc() { - return src; - } - - public ECSchema getSchema() { - return schema; - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java deleted file mode 100644 index 282eeaf60bd88..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.hdfs.protocol; - -import org.apache.hadoop.io.erasurecode.ECSchema; - -/** - * Information about the EC Zone at the specified path. - */ -public class ErasureCodingZoneInfo { - - private String dir; - private ECSchema schema; - private int cellSize; - - public ErasureCodingZoneInfo(String dir, ECSchema schema, int cellSize) { - this.dir = dir; - this.schema = schema; - this.cellSize = cellSize; - } - - /** - * Get directory of the EC zone. - * - * @return - */ - public String getDir() { - return dir; - } - - /** - * Get the schema for the EC Zone - * - * @return - */ - public ECSchema getSchema() { - return schema; - } - - /** - * Get cellSize for the EC Zone - */ - public int getCellSize() { - return cellSize; - } - - @Override - public String toString() { - return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: " - + cellSize; - } -} 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 863b217254dac..4228a65838a78 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 @@ -35,8 +35,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -202,10 +201,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto; @@ -1524,22 +1521,6 @@ public GetEditsFromTxidResponseProto getEditsFromTxid(RpcController controller, } } - @Override - public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller, - GetErasureCodingInfoRequestProto request) throws ServiceException { - try { - ErasureCodingInfo ecInfo = server.getErasureCodingInfo(request.getSrc()); - GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto - .newBuilder(); - if (ecInfo != null) { - resBuilder.setECInfo(PBHelper.convertECInfo(ecInfo)); - } - return resBuilder.build(); - } catch (IOException e) { - throw new ServiceException(e); - } - } - @Override public GetECSchemasResponseProto getECSchemas(RpcController controller, GetECSchemasRequestProto request) throws ServiceException { @@ -1557,13 +1538,13 @@ public GetECSchemasResponseProto getECSchemas(RpcController controller, } @Override - public GetErasureCodingZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller, - GetErasureCodingZoneInfoRequestProto request) throws ServiceException { + public GetErasureCodingZoneResponseProto getErasureCodingZone(RpcController controller, + GetErasureCodingZoneRequestProto request) throws ServiceException { try { - ErasureCodingZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc()); - GetErasureCodingZoneInfoResponseProto.Builder builder = GetErasureCodingZoneInfoResponseProto.newBuilder(); - if (ecZoneInfo != null) { - builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo)); + ErasureCodingZone ecZone = server.getErasureCodingZone(request.getSrc()); + GetErasureCodingZoneResponseProto.Builder builder = GetErasureCodingZoneResponseProto.newBuilder(); + if (ecZone != null) { + builder.setECZone(PBHelper.convertErasureCodingZone(ecZone)); } return builder.build(); } catch (IOException 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 336e3a25a8ea4..2c88a93e3d55b 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 @@ -58,8 +58,7 @@ 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.ErasureCodingInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -168,10 +167,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; @@ -1552,21 +1549,6 @@ public EventBatchList getEditsFromTxid(long txid) throws IOException { } } - @Override - public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException { - GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder() - .setSrc(src).build(); - try { - GetErasureCodingInfoResponseProto res = rpcProxy.getErasureCodingInfo(null, req); - if (res.hasECInfo()) { - return PBHelper.convertECInfo(res.getECInfo()); - } - return null; - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } - } - @Override public ECSchema[] getECSchemas() throws IOException { try { @@ -1584,14 +1566,14 @@ public ECSchema[] getECSchemas() throws IOException { } @Override - public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException { - GetErasureCodingZoneInfoRequestProto req = GetErasureCodingZoneInfoRequestProto.newBuilder() + public ErasureCodingZone getErasureCodingZone(String src) throws IOException { + GetErasureCodingZoneRequestProto req = GetErasureCodingZoneRequestProto.newBuilder() .setSrc(src).build(); try { - GetErasureCodingZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo( + GetErasureCodingZoneResponseProto response = rpcProxy.getErasureCodingZone( null, req); - if (response.hasECZoneInfo()) { - return PBHelper.convertECZoneInfo(response.getECZoneInfo()); + if (response.hasECZone()) { + return PBHelper.convertErasureCodingZone(response.getECZone()); } return null; } catch (ServiceException 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 4d0f8710cf239..b2415fa83bf2b 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 @@ -77,13 +77,12 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; -import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -135,8 +134,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto; @@ -203,7 +201,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand; @@ -3123,16 +3120,6 @@ public static BlockReportContextProto convert(BlockReportContext context) { build(); } - public static ErasureCodingInfo convertECInfo(ErasureCodingInfoProto ecInfoProto) { - return new ErasureCodingInfo(ecInfoProto.getSrc(), - convertECSchema(ecInfoProto.getSchema())); - } - - public static ErasureCodingInfoProto convertECInfo(ErasureCodingInfo ecInfo) { - return ErasureCodingInfoProto.newBuilder().setSrc(ecInfo.getSrc()) - .setSchema(convertECSchema(ecInfo.getSchema())).build(); - } - public static ECSchema convertECSchema(ECSchemaProto schema) { List optionsList = schema.getOptionsList(); Map options = new HashMap<>(optionsList.size()); @@ -3157,16 +3144,17 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { return builder.build(); } - public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) { - return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir()) - .setSchema(convertECSchema(ecZoneInfo.getSchema())) - .setCellSize(ecZoneInfo.getCellSize()).build(); + public static ErasureCodingZoneProto convertErasureCodingZone( + ErasureCodingZone ecZone) { + return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir()) + .setSchema(convertECSchema(ecZone.getSchema())) + .setCellSize(ecZone.getCellSize()).build(); } - public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) { - return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(), - convertECSchema(ecZoneInfoProto.getSchema()), - ecZoneInfoProto.getCellSize()); + public static ErasureCodingZone convertErasureCodingZone( + ErasureCodingZoneProto ecZoneProto) { + return new ErasureCodingZone(ecZoneProto.getDir(), + convertECSchema(ecZoneProto.getSchema()), ecZoneProto.getCellSize()); } public static BlockECRecoveryInfo convertBlockECRecoveryInfo( 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 79cbcc6e8db01..8f1f6b71c951e 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 @@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -1555,14 +1555,14 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { assert rw instanceof ErasureCodingWork; assert rw.targets.length > 0; String src = block.getBlockCollection().getName(); - ErasureCodingZoneInfo ecZoneInfo = null; + ErasureCodingZone ecZone = null; try { - ecZoneInfo = namesystem.getErasureCodingZoneInfoForPath(src); + ecZone = namesystem.getErasureCodingZoneForPath(src); } catch (IOException e) { blockLog - .warn("Failed to get the EC zone info for the file {} ", src); + .warn("Failed to get the EC zone for the file {} ", src); } - if (ecZoneInfo == null) { + if (ecZone == null) { blockLog.warn("No EC schema found for the file {}. " + "So cannot proceed for recovery", src); // TODO: we may have to revisit later for what we can do better to @@ -1573,7 +1573,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { new ExtendedBlock(namesystem.getBlockPoolId(), block), rw.srcNodes, rw.targets, ((ErasureCodingWork) rw).liveBlockIndicies, - ecZoneInfo.getSchema(), ecZoneInfo.getCellSize()); + ecZone.getSchema(), ecZone.getCellSize()); } else { rw.srcNodes[0].addBlockToBeReplicated(block, targets); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 89fecc612c321..4b3e747927b87 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.WritableUtils; @@ -61,11 +61,11 @@ public ErasureCodingZoneManager(FSDirectory dir) { } ECSchema getECSchema(INodesInPath iip) throws IOException { - ErasureCodingZoneInfo ecZoneInfo = getECZoneInfo(iip); - return ecZoneInfo == null ? null : ecZoneInfo.getSchema(); + ErasureCodingZone ecZone = getECZone(iip); + return ecZone == null ? null : ecZone.getSchema(); } - ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { + ErasureCodingZone getECZone(INodesInPath iip) throws IOException { assert dir.hasReadLock(); Preconditions.checkNotNull(iip); List inodes = iip.getReadOnlyINodes(); @@ -92,7 +92,7 @@ ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { String schemaName = WritableUtils.readString(dIn); ECSchema schema = dir.getFSNamesystem().getECSchemaManager() .getSchema(schemaName); - return new ErasureCodingZoneInfo(dir.getInode(inode.getId()) + return new ErasureCodingZone(dir.getInode(inode.getId()) .getFullPathName(), schema, cellSize); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index eba5013a5a5e5..1b1d79ef480df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -386,9 +386,9 @@ static HdfsFileStatus createFileStatus( final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); - final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip); - final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null; - final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0; + final ErasureCodingZone ecZone = fsd.getECZone(iip); + final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; + final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; if (node.isFile()) { final INodeFile fileNode = node.asFile(); @@ -468,9 +468,9 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip); - final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null; - final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0; + final ErasureCodingZone ecZone = fsd.getECZone(iip); + final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; + final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(size, node.isDirectory(), replication, 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 735ea5a928ed6..0e3694f31b1ca 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 @@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException; import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException; @@ -1253,10 +1253,10 @@ ECSchema getECSchema(INodesInPath iip) throws IOException { } } - ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException { + ErasureCodingZone getECZone(INodesInPath iip) throws IOException { readLock(); try { - return ecZoneManager.getECZoneInfo(iip); + return ecZoneManager.getECZone(iip); } finally { readUnlock(); } 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 a53cd61b20ec8..59c9a03fbe919 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 @@ -181,8 +181,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.EncryptionZone; -import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -7623,28 +7622,16 @@ private boolean isInECZone(String src) throws IOException { return dir.isInECZone(iip); } - /** - * Get the erasure coding information for specified src - */ - ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException, - UnresolvedLinkException, IOException { - ErasureCodingZoneInfo zoneInfo = getErasureCodingZoneInfo(src); - if (zoneInfo != null) { - return new ErasureCodingInfo(src, zoneInfo.getSchema()); - } - return null; - } - /** * Get the erasure coding zone information for specified path */ - ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) + ErasureCodingZone getErasureCodingZone(String src) throws AccessControlException, UnresolvedLinkException, IOException { checkOperation(OperationCategory.READ); readLock(); try { checkOperation(OperationCategory.READ); - return getErasureCodingZoneInfoForPath(src); + return getErasureCodingZoneForPath(src); } finally { readUnlock(); } @@ -7865,7 +7852,7 @@ private static void enableAsyncAuditLog() { } @Override - public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src) + public ErasureCodingZone getErasureCodingZoneForPath(String src) throws IOException { final byte[][] pathComponents = FSDirectory .getPathComponentsForReservedPath(src); @@ -7875,7 +7862,7 @@ public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src) if (isPermissionEnabled) { dir.checkPathAccess(pc, iip, FsAction.READ); } - return dir.getECZoneInfo(iip); + return dir.getECZone(iip); } } 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 9e94b90863e9e..1377bbed4f920 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 @@ -84,8 +84,7 @@ 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.ErasureCodingInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FSLimitException; @@ -2035,12 +2034,6 @@ public void removeSpanReceiver(long id) throws IOException { nn.spanReceiverHost.removeSpanReceiver(id); } - @Override // ClientProtocol - public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException { - checkNNStartup(); - return namesystem.getErasureCodingInfo(src); - } - @Override // ClientProtocol public ECSchema[] getECSchemas() throws IOException { checkNNStartup(); @@ -2048,8 +2041,8 @@ public ECSchema[] getECSchemas() throws IOException { } @Override // ClientProtocol - public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException { + public ErasureCodingZone getErasureCodingZone(String src) throws IOException { checkNNStartup(); - return namesystem.getErasureCodingZoneInfo(src); + return namesystem.getErasureCodingZone(src); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java index a32e800e5129f..2a9ab39b76ea7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java @@ -21,11 +21,10 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.util.RwLock; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.StandbyException; import org.apache.hadoop.security.AccessControlException; @@ -52,13 +51,12 @@ public interface Namesystem extends RwLock, SafeMode { public boolean isInSnapshot(BlockCollection bc); /** - * Gets the ECZone info for path - * + * Gets the ECZone for path * @param src * - path - * @return {@link ErasureCodingZoneInfo} + * @return {@link ErasureCodingZone} * @throws IOException */ - public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src) + public ErasureCodingZone getErasureCodingZoneForPath(String src) throws IOException; } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index 2d8220851f118..34965d29c7c0d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.shell.CommandFactory; import org.apache.hadoop.fs.shell.PathData; import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -47,8 +47,8 @@ public static void registerCommands(CommandFactory factory) { // Register all commands of Erasure CLI, with a '-' at the beginning in name // of the command. factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME); - factory.addClass(GetECZoneInfoCommand.class, "-" - + GetECZoneInfoCommand.NAME); + factory.addClass(GetECZoneCommand.class, "-" + + GetECZoneCommand.NAME); factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME); } @@ -153,8 +153,8 @@ protected void processPath(PathData item) throws IOException { /** * Get the information about the zone */ - static class GetECZoneInfoCommand extends ECCommand { - public static final String NAME = "getZoneInfo"; + static class GetECZoneCommand extends ECCommand { + public static final String NAME = "getZone"; public static final String USAGE = ""; public static final String DESCRIPTION = "Get information about the EC zone at specified path\n"; @@ -174,9 +174,9 @@ protected void processPath(PathData item) throws IOException { super.processPath(item); DistributedFileSystem dfs = (DistributedFileSystem) item.fs; try { - ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path); - if (ecZoneInfo != null) { - out.println(ecZoneInfo.toString()); + ErasureCodingZone ecZone = dfs.getErasureCodingZone(item.path); + if (ecZone != null) { + out.println(ecZone.toString()); } else { out.println("Path " + item.path + " is not in EC zone"); } 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 0a0a4c4c934c1..2b64ca0d4c9a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -863,10 +863,8 @@ service ClientNamenodeProtocol { returns(GetCurrentEditLogTxidResponseProto); rpc getEditsFromTxid(GetEditsFromTxidRequestProto) returns(GetEditsFromTxidResponseProto); - rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto) - returns(GetErasureCodingInfoResponseProto); rpc getECSchemas(GetECSchemasRequestProto) returns(GetECSchemasResponseProto); - rpc getErasureCodingZoneInfo(GetErasureCodingZoneInfoRequestProto) - returns(GetErasureCodingZoneInfoResponseProto); + rpc getErasureCodingZone(GetErasureCodingZoneRequestProto) + returns(GetErasureCodingZoneResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 058ed96ee3a20..56bb7a2f2961c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -24,17 +24,9 @@ package hadoop.hdfs; import "hdfs.proto"; /** - * ErasureCodingInfo + * ErasureCodingZone */ -message ErasureCodingInfoProto { - required string src = 1; - required ECSchemaProto schema = 2; -} - -/** - * ErasureCodingZoneInfo - */ -message ErasureCodingZoneInfoProto { +message ErasureCodingZoneProto { required string dir = 1; required ECSchemaProto schema = 2; required uint32 cellSize = 3; @@ -49,14 +41,6 @@ message CreateErasureCodingZoneRequestProto { message CreateErasureCodingZoneResponseProto { } -message GetErasureCodingInfoRequestProto { - required string src = 1; -} - -message GetErasureCodingInfoResponseProto { - optional ErasureCodingInfoProto ECInfo = 1; -} - message GetECSchemasRequestProto { // void request } @@ -64,12 +48,12 @@ message GetECSchemasResponseProto { repeated ECSchemaProto schemas = 1; } -message GetErasureCodingZoneInfoRequestProto { +message GetErasureCodingZoneRequestProto { required string src = 1; // path to get the zone info } -message GetErasureCodingZoneInfoResponseProto { - optional ErasureCodingZoneInfoProto ECZoneInfo = 1; +message GetErasureCodingZoneResponseProto { + optional ErasureCodingZoneProto ECZone = 1; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index 480791ee199e9..d724b53d6fbf7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INode; @@ -177,12 +177,12 @@ public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); // dir ECInfo before creating ec zone - assertNull(fs.getClient().getErasureCodingInfo(src)); + assertNull(fs.getClient().getFileInfo(src).getECSchema()); // dir ECInfo after creating ec zone fs.getClient().createErasureCodingZone(src, null, 0); //Default one will be used. ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); verifyErasureCodingInfo(src, sysDefaultSchema); - fs.create(new Path(ecDir, "/child1")).close(); + fs.create(new Path(ecDir, "child1")).close(); // verify for the files in ec zone verifyErasureCodingInfo(src + "/child1", sysDefaultSchema); } @@ -198,21 +198,19 @@ public void testGetErasureCodingInfo() throws Exception { final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); // dir ECInfo before creating ec zone - assertNull(fs.getClient().getErasureCodingInfo(src)); + assertNull(fs.getClient().getFileInfo(src).getECSchema()); // dir ECInfo after creating ec zone fs.getClient().createErasureCodingZone(src, usingSchema, 0); verifyErasureCodingInfo(src, usingSchema); - fs.create(new Path(ecDir, "/child1")).close(); + fs.create(new Path(ecDir, "child1")).close(); // verify for the files in ec zone verifyErasureCodingInfo(src + "/child1", usingSchema); } private void verifyErasureCodingInfo( String src, ECSchema usingSchema) throws IOException { - ErasureCodingInfo ecInfo = fs.getClient().getErasureCodingInfo(src); - assertNotNull("ECInfo should have been non-null", ecInfo); - assertEquals(src, ecInfo.getSrc()); - ECSchema schema = ecInfo.getSchema(); + HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src); + ECSchema schema = hdfsFileStatus.getECSchema(); assertNotNull(schema); assertEquals("Actually used schema should be equal with target schema", usingSchema, schema); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml index 66892f0a019a0..ee1a19aec2d49 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -67,9 +67,9 @@ - help: getZoneInfo command + help: getZone command - -fs NAMENODE -help getZoneInfo + -fs NAMENODE -help getZone @@ -80,7 +80,7 @@ RegexpComparator - ^-getZoneInfo <path>(.)* + ^-getZone <path>(.)* @@ -145,7 +145,7 @@ -fs NAMENODE -mkdir /eczone -fs NAMENODE -createZone /eczone - -fs NAMENODE -getZoneInfo /eczone + -fs NAMENODE -getZone /eczone -fs NAMENODE -rmdir /eczone @@ -159,10 +159,10 @@ - getZoneInfo : get information about the EC zone at specified path not in zone + getZone : get information about the EC zone at specified path not in zone -fs NAMENODE -mkdir /noec - -fs NAMENODE -getZoneInfo /noec + -fs NAMENODE -getZone /noec -fs NAMENODE -rmdir /noec @@ -176,11 +176,11 @@ - getZoneInfo : get information about the EC zone at specified path + getZone : get information about the EC zone at specified path -fs NAMENODE -mkdir /eczone -fs NAMENODE -createZone -s RS-6-3 /eczone - -fs NAMENODE -getZoneInfo /eczone + -fs NAMENODE -getZone /eczone -fs NAMENODE -rmdir /eczone @@ -194,12 +194,12 @@ - getZoneInfo : get EC zone at specified file path + getZone : get EC zone at specified file path -fs NAMENODE -mkdir /eczone -fs NAMENODE -createZone -s RS-6-3 /eczone -fs NAMENODE -touchz /eczone/ecfile - -fs NAMENODE -getZoneInfo /eczone/ecfile + -fs NAMENODE -getZone /eczone/ecfile -fs NAMENODE -rm /eczone/ecfile @@ -313,24 +313,24 @@ - getZoneInfo : illegal parameters - path is missing + getZone : illegal parameters - path is missing - -fs NAMENODE -getZoneInfo + -fs NAMENODE -getZone RegexpComparator - ^-getZoneInfo: <path> is missing(.)* + ^-getZone: <path> is missing(.)* - getZoneInfo : illegal parameters - too many arguments + getZone : illegal parameters - too many arguments - -fs NAMENODE -getZoneInfo /eczone /eczone + -fs NAMENODE -getZone /eczone /eczone -fs NAMENODE -rm /eczone @@ -338,22 +338,22 @@ SubstringComparator - -getZoneInfo: Too many arguments + -getZone: Too many arguments - getZoneInfo : illegal parameters - no such file + getZone : illegal parameters - no such file - -fs NAMENODE -getZoneInfo /eczone + -fs NAMENODE -getZone /eczone RegexpComparator - ^getZoneInfo: `/eczone': No such file or directory(.)* + ^getZone: `/eczone': No such file or directory(.)* From 80c56c2d3167f5195c72802071ea6dfb52abff1a Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 25 May 2015 13:24:50 +0530 Subject: [PATCH 137/212] HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B) Added missed file --- .../hdfs/protocol/ErasureCodingZone.java | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java new file mode 100644 index 0000000000000..655def3da53d4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * Information about the EC Zone at the specified path. + */ +public class ErasureCodingZone { + + private String dir; + private ECSchema schema; + private int cellSize; + + public ErasureCodingZone(String dir, ECSchema schema, int cellSize) { + this.dir = dir; + this.schema = schema; + this.cellSize = cellSize; + } + + /** + * Get directory of the EC zone. + * + * @return + */ + public String getDir() { + return dir; + } + + /** + * Get the schema for the EC Zone + * + * @return + */ + public ECSchema getSchema() { + return schema; + } + + /** + * Get cellSize for the EC Zone + */ + public int getCellSize() { + return cellSize; + } + + @Override + public String toString() { + return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: " + + cellSize; + } +} From 5a391e1d2584dc5d77fb1325ea91c8e5854934d1 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 25 May 2015 16:02:37 +0530 Subject: [PATCH 138/212] HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders (Contributed by Vinayakumar B) --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 5 ++- .../hadoop/io/erasurecode/ECSchema.java | 40 +------------------ .../hadoop/io/erasurecode/TestECSchema.java | 3 -- 3 files changed, 6 insertions(+), 42 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 531b8d56ae24d..c9b80d3bba58d 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -55,4 +55,7 @@ HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure coder. (Kai Zheng via Zhe Zhang) - HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng) \ No newline at end of file + HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng) + + HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders + (vinayakumarb) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index f058ea71888ed..fdc569ee9a9ff 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -28,8 +28,6 @@ public final class ECSchema { public static final String NUM_DATA_UNITS_KEY = "k"; public static final String NUM_PARITY_UNITS_KEY = "m"; public static final String CODEC_NAME_KEY = "codec"; - public static final String CHUNK_SIZE_KEY = "chunkSize"; - public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K /** * A friendly and understandable name that can mean what's it, also serves as @@ -52,11 +50,6 @@ public final class ECSchema { */ private final int numParityUnits; - /** - * Unit data size for each chunk in a coding - */ - private final int chunkSize; - /* * An erasure code can have its own specific advanced parameters, subject to * itself to interpret these key-value settings. @@ -92,17 +85,9 @@ public ECSchema(String schemaName, Map allOptions) { this.numDataUnits = tmpNumDataUnits; this.numParityUnits = tmpNumParityUnits; - int tmpChunkSize = extractIntOption(CHUNK_SIZE_KEY, allOptions); - if (tmpChunkSize > 0) { - this.chunkSize = tmpChunkSize; - } else { - this.chunkSize = DEFAULT_CHUNK_SIZE; - } - allOptions.remove(CODEC_NAME_KEY); allOptions.remove(NUM_DATA_UNITS_KEY); allOptions.remove(NUM_PARITY_UNITS_KEY); - allOptions.remove(CHUNK_SIZE_KEY); // After some cleanup this.extraOptions = Collections.unmodifiableMap(allOptions); } @@ -144,14 +129,6 @@ public ECSchema(String schemaName, String codecName, int numDataUnits, extraOptions = new HashMap<>(); } - int tmpChunkSize = extractIntOption(CHUNK_SIZE_KEY, extraOptions); - if (tmpChunkSize > 0) { - this.chunkSize = tmpChunkSize; - } else { - this.chunkSize = DEFAULT_CHUNK_SIZE; - } - - extraOptions.remove(CHUNK_SIZE_KEY); // After some cleanup this.extraOptions = Collections.unmodifiableMap(extraOptions); } @@ -216,14 +193,6 @@ public int getNumParityUnits() { return numParityUnits; } - /** - * Get chunk buffer size for the erasure encoding/decoding. - * @return chunk buffer size - */ - public int getChunkSize() { - return chunkSize; - } - /** * Make a meaningful string representation for log output. * @return string representation @@ -235,9 +204,8 @@ public String toString() { sb.append("Name=" + schemaName + ", "); sb.append("Codec=" + codecName + ", "); sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ", "); - sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ", "); - sb.append(CHUNK_SIZE_KEY + "=" + chunkSize + - (extraOptions.isEmpty() ? "" : ", ")); + sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits); + sb.append((extraOptions.isEmpty() ? "" : ", ")); int i = 0; for (String opt : extraOptions.keySet()) { @@ -267,9 +235,6 @@ public boolean equals(Object o) { if (numParityUnits != ecSchema.numParityUnits) { return false; } - if (chunkSize != ecSchema.chunkSize) { - return false; - } if (!schemaName.equals(ecSchema.schemaName)) { return false; } @@ -286,7 +251,6 @@ public int hashCode() { result = 31 * result + extraOptions.hashCode(); result = 31 * result + numDataUnits; result = 31 * result + numParityUnits; - result = 31 * result + chunkSize; return result; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java index 15e672fbb9fed..c362b9619e1a4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java @@ -29,7 +29,6 @@ public void testGoodSchema() { String schemaName = "goodSchema"; int numDataUnits = 6; int numParityUnits = 3; - int chunkSize = 64 * 1024 * 1024; String codec = "rs"; String extraOption = "extraOption"; String extraOptionValue = "extraOptionValue"; @@ -38,7 +37,6 @@ public void testGoodSchema() { options.put(ECSchema.NUM_DATA_UNITS_KEY, String.valueOf(numDataUnits)); options.put(ECSchema.NUM_PARITY_UNITS_KEY, String.valueOf(numParityUnits)); options.put(ECSchema.CODEC_NAME_KEY, codec); - options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(chunkSize)); options.put(extraOption, extraOptionValue); ECSchema schema = new ECSchema(schemaName, options); @@ -47,7 +45,6 @@ public void testGoodSchema() { assertEquals(schemaName, schema.getSchemaName()); assertEquals(numDataUnits, schema.getNumDataUnits()); assertEquals(numParityUnits, schema.getNumParityUnits()); - assertEquals(chunkSize, schema.getChunkSize()); assertEquals(codec, schema.getCodecName()); assertEquals(extraOptionValue, schema.getExtraOptions().get(extraOption)); } From 4ad484883f773c702a1874fc12816ef1a4a54136 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Tue, 26 May 2015 22:45:19 +0800 Subject: [PATCH 139/212] HADOOP-11847 Enhance raw coder allowing to read least required inputs in decoding. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 + .../rawcoder/AbstractRawErasureCoder.java | 27 ++- .../rawcoder/AbstractRawErasureDecoder.java | 75 ++++++-- .../rawcoder/AbstractRawErasureEncoder.java | 8 +- .../io/erasurecode/rawcoder/RSRawDecoder.java | 162 +++++++++++++++++- .../rawcoder/RawErasureDecoder.java | 20 ++- .../erasurecode/rawcoder/XORRawDecoder.java | 2 +- .../erasurecode/rawcoder/XORRawEncoder.java | 2 +- .../rawcoder/util/GaloisField.java | 12 +- .../hadoop/io/erasurecode/TestCoderBase.java | 39 ++--- .../coder/TestErasureCoderBase.java | 1 - .../erasurecode/rawcoder/TestRSRawCoder.java | 101 ++++++----- .../rawcoder/TestRawCoderBase.java | 54 ++++++ .../erasurecode/rawcoder/TestXORRawCoder.java | 45 ++--- 14 files changed, 397 insertions(+), 154 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index c9b80d3bba58d..0c244737849d1 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -59,3 +59,6 @@ HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders (vinayakumarb) + + HADOOP-11847. Enhance raw coder allowing to read least required inputs in decoding. + (Kai Zheng) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index 06ae660f36728..e6a1542b4035c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -60,12 +60,13 @@ public void release() { } /** - * Ensure output buffer filled with ZERO bytes fully in chunkSize. - * @param buffer a buffer ready to write chunk size bytes + * Ensure a buffer filled with ZERO bytes from current readable/writable + * position. + * @param buffer a buffer ready to read / write certain size bytes * @return the buffer itself, with ZERO bytes written, the position and limit * are not changed after the call */ - protected ByteBuffer resetOutputBuffer(ByteBuffer buffer) { + protected ByteBuffer resetBuffer(ByteBuffer buffer) { int pos = buffer.position(); for (int i = pos; i < buffer.limit(); ++i) { buffer.put((byte) 0); @@ -77,7 +78,7 @@ protected ByteBuffer resetOutputBuffer(ByteBuffer buffer) { /** * Ensure the buffer (either input or output) ready to read or write with ZERO - * bytes fully in chunkSize. + * bytes fully in specified length of len. * @param buffer bytes array buffer * @return the buffer itself */ @@ -92,11 +93,16 @@ protected byte[] resetBuffer(byte[] buffer, int offset, int len) { /** * Check and ensure the buffers are of the length specified by dataLen. * @param buffers + * @param allowNull * @param dataLen */ - protected void ensureLength(ByteBuffer[] buffers, int dataLen) { + protected void ensureLength(ByteBuffer[] buffers, + boolean allowNull, int dataLen) { for (int i = 0; i < buffers.length; ++i) { - if (buffers[i].remaining() != dataLen) { + if (buffers[i] == null && !allowNull) { + throw new HadoopIllegalArgumentException( + "Invalid buffer found, not allowing null"); + } else if (buffers[i] != null && buffers[i].remaining() != dataLen) { throw new HadoopIllegalArgumentException( "Invalid buffer, not of length " + dataLen); } @@ -106,11 +112,16 @@ protected void ensureLength(ByteBuffer[] buffers, int dataLen) { /** * Check and ensure the buffers are of the length specified by dataLen. * @param buffers + * @param allowNull * @param dataLen */ - protected void ensureLength(byte[][] buffers, int dataLen) { + protected void ensureLength(byte[][] buffers, + boolean allowNull, int dataLen) { for (int i = 0; i < buffers.length; ++i) { - if (buffers[i].length != dataLen) { + if (buffers[i] == null && !allowNull) { + throw new HadoopIllegalArgumentException( + "Invalid buffer found, not allowing null"); + } else if (buffers[i] != null && buffers[i].length != dataLen) { throw new HadoopIllegalArgumentException( "Invalid buffer not of length " + dataLen); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java index 0c1f80f968380..c6105b0921cd7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -21,6 +21,7 @@ import org.apache.hadoop.io.erasurecode.ECChunk; import java.nio.ByteBuffer; +import java.util.Arrays; /** * An abstract raw erasure decoder that's to be inherited by new decoders. @@ -38,14 +39,16 @@ public AbstractRawErasureDecoder(int numDataUnits, int numParityUnits) { public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { checkParameters(inputs, erasedIndexes, outputs); - int dataLen = inputs[0].remaining(); + + ByteBuffer validInput = findFirstValidInput(inputs); + int dataLen = validInput.remaining(); if (dataLen == 0) { return; } - ensureLength(inputs, dataLen); - ensureLength(outputs, dataLen); + ensureLength(inputs, true, dataLen); + ensureLength(outputs, false, dataLen); - boolean usingDirectBuffer = inputs[0].isDirect(); + boolean usingDirectBuffer = validInput.isDirect(); if (usingDirectBuffer) { doDecode(inputs, erasedIndexes, outputs); return; @@ -59,8 +62,10 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer buffer; for (int i = 0; i < inputs.length; ++i) { buffer = inputs[i]; - inputOffsets[i] = buffer.position(); - newInputs[i] = buffer.array(); + if (buffer != null) { + inputOffsets[i] = buffer.position(); + newInputs[i] = buffer.array(); + } } for (int i = 0; i < outputs.length; ++i) { @@ -74,7 +79,10 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, for (int i = 0; i < inputs.length; ++i) { buffer = inputs[i]; - buffer.position(inputOffsets[i] + dataLen); // dataLen bytes consumed + if (buffer != null) { + // dataLen bytes consumed + buffer.position(inputOffsets[i] + dataLen); + } } } @@ -90,12 +98,14 @@ protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, @Override public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) { checkParameters(inputs, erasedIndexes, outputs); - int dataLen = inputs[0].length; + + byte[] validInput = findFirstValidInput(inputs); + int dataLen = validInput.length; if (dataLen == 0) { return; } - ensureLength(inputs, dataLen); - ensureLength(outputs, dataLen); + ensureLength(inputs, true, dataLen); + ensureLength(outputs, false, dataLen); int[] inputOffsets = new int[inputs.length]; // ALL ZERO int[] outputOffsets = new int[outputs.length]; // ALL ZERO @@ -148,5 +158,50 @@ protected void checkParameters(Object[] inputs, int[] erasedIndexes, throw new HadoopIllegalArgumentException( "Too many erased, not recoverable"); } + + int validInputs = 0; + for (int i = 0; i < inputs.length; ++i) { + if (inputs[i] != null) { + validInputs += 1; + } + } + + if (validInputs < getNumDataUnits()) { + throw new HadoopIllegalArgumentException( + "No enough valid inputs are provided, not recoverable"); + } + } + + /** + * Get indexes into inputs array for items marked as null, either erased or + * not to read. + * @return indexes into inputs array + */ + protected int[] getErasedOrNotToReadIndexes(Object[] inputs) { + int[] invalidIndexes = new int[inputs.length]; + int idx = 0; + for (int i = 0; i < inputs.length; i++) { + if (inputs[i] == null) { + invalidIndexes[idx++] = i; + } + } + + return Arrays.copyOf(invalidIndexes, idx); + } + + /** + * Find the valid input from all the inputs. + * @param inputs + * @return the first valid input + */ + protected static T findFirstValidInput(T[] inputs) { + for (int i = 0; i < inputs.length; i++) { + if (inputs[i] != null) { + return inputs[i]; + } + } + + throw new HadoopIllegalArgumentException( + "Invalid inputs are found, all being null"); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java index c7a136b0f4f99..d1faa8c65bebe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -41,8 +41,8 @@ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { if (dataLen == 0) { return; } - ensureLength(inputs, dataLen); - ensureLength(outputs, dataLen); + ensureLength(inputs, false, dataLen); + ensureLength(outputs, false, dataLen); boolean usingDirectBuffer = inputs[0].isDirect(); if (usingDirectBuffer) { @@ -90,8 +90,8 @@ public void encode(byte[][] inputs, byte[][] outputs) { if (dataLen == 0) { return; } - ensureLength(inputs, dataLen); - ensureLength(outputs, dataLen); + ensureLength(inputs, false, dataLen); + ensureLength(outputs, false, dataLen); int[] inputOffsets = new int[inputs.length]; // ALL ZERO int[] outputOffsets = new int[outputs.length]; // ALL ZERO diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java index e265dcef5d395..57e6957435ca3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.io.erasurecode.rawcoder; +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil; import java.nio.ByteBuffer; @@ -25,35 +26,64 @@ * A raw erasure decoder in RS code scheme in pure Java in case native one * isn't available in some environment. Please always use native implementations * when possible. + * + * Currently this implementation will compute and decode not to read units + * unnecessarily due to the underlying implementation limit in GF. This will be + * addressed in HADOOP-11871. */ public class RSRawDecoder extends AbstractRawErasureDecoder { // To describe and calculate the needed Vandermonde matrix private int[] errSignature; private int[] primitivePower; + /** + * We need a set of reusable buffers either for the bytes array + * decoding version or direct buffer decoding version. Normally not both. + * + * For output, in addition to the valid buffers from the caller + * passed from above, we need to provide extra buffers for the internal + * decoding implementation. For output, the caller should provide no more + * than numParityUnits but at least one buffers. And the left buffers will be + * borrowed from either bytesArrayBuffers, for the bytes array version. + * + */ + // Reused buffers for decoding with bytes arrays + private byte[][] bytesArrayBuffers = new byte[getNumParityUnits()][]; + private byte[][] adjustedByteArrayOutputsParameter = + new byte[getNumParityUnits()][]; + private int[] adjustedOutputOffsets = new int[getNumParityUnits()]; + + // Reused buffers for decoding with direct ByteBuffers + private ByteBuffer[] directBuffers = new ByteBuffer[getNumParityUnits()]; + private ByteBuffer[] adjustedDirectBufferOutputsParameter = + new ByteBuffer[getNumParityUnits()]; + public RSRawDecoder(int numDataUnits, int numParityUnits) { super(numDataUnits, numParityUnits); - assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); + if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) { + throw new HadoopIllegalArgumentException( + "Invalid numDataUnits and numParityUnits"); + } this.errSignature = new int[numParityUnits]; this.primitivePower = RSUtil.getPrimitivePower(numDataUnits, numParityUnits); } - @Override - protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + private void doDecodeImpl(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { + ByteBuffer valid = findFirstValidInput(inputs); + int dataLen = valid.remaining(); for (int i = 0; i < erasedIndexes.length; i++) { errSignature[i] = primitivePower[erasedIndexes[i]]; - RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]); + RSUtil.GF.substitute(inputs, dataLen, outputs[i], primitivePower[i]); } RSUtil.GF.solveVandermondeSystem(errSignature, outputs, erasedIndexes.length); } - @Override - protected void doDecode(byte[][] inputs, int[] inputOffsets, + private void doDecodeImpl(byte[][] inputs, int[] inputOffsets, int dataLen, int[] erasedIndexes, byte[][] outputs, int[] outputOffsets) { for (int i = 0; i < erasedIndexes.length; i++) { @@ -63,6 +93,124 @@ protected void doDecode(byte[][] inputs, int[] inputOffsets, } RSUtil.GF.solveVandermondeSystem(errSignature, outputs, outputOffsets, - erasedIndexes.length, dataLen); + erasedIndexes.length, dataLen); + } + + @Override + protected void doDecode(byte[][] inputs, int[] inputOffsets, + int dataLen, int[] erasedIndexes, + byte[][] outputs, int[] outputOffsets) { + /** + * As passed parameters are friendly to callers but not to the underlying + * implementations, so we have to adjust them before calling doDecodeImpl. + */ + + int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs); + + // Prepare for adjustedOutputsParameter + + // First reset the positions needed this time + for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) { + adjustedByteArrayOutputsParameter[i] = null; + adjustedOutputOffsets[i] = 0; + } + // Use the caller passed buffers in erasedIndexes positions + for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) { + boolean found = false; + for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) { + // If this index is one requested by the caller via erasedIndexes, then + // we use the passed output buffer to avoid copying data thereafter. + if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) { + found = true; + adjustedByteArrayOutputsParameter[j] = resetBuffer( + outputs[outputIdx], outputOffsets[outputIdx], dataLen); + adjustedOutputOffsets[j] = outputOffsets[outputIdx]; + outputIdx++; + } + } + if (!found) { + throw new HadoopIllegalArgumentException( + "Inputs not fully corresponding to erasedIndexes in null places"); + } + } + // Use shared buffers for other positions (not set yet) + for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) { + if (adjustedByteArrayOutputsParameter[i] == null) { + adjustedByteArrayOutputsParameter[i] = resetBuffer( + checkGetBytesArrayBuffer(bufferIdx, dataLen), 0, dataLen); + adjustedOutputOffsets[i] = 0; // Always 0 for such temp output + bufferIdx++; + } + } + + doDecodeImpl(inputs, inputOffsets, dataLen, erasedOrNotToReadIndexes, + adjustedByteArrayOutputsParameter, adjustedOutputOffsets); + } + + @Override + protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + ByteBuffer validInput = findFirstValidInput(inputs); + int dataLen = validInput.remaining(); + + /** + * As passed parameters are friendly to callers but not to the underlying + * implementations, so we have to adjust them before calling doDecodeImpl. + */ + + int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs); + + // Prepare for adjustedDirectBufferOutputsParameter + + // First reset the positions needed this time + for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) { + adjustedDirectBufferOutputsParameter[i] = null; + } + // Use the caller passed buffers in erasedIndexes positions + for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) { + boolean found = false; + for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) { + // If this index is one requested by the caller via erasedIndexes, then + // we use the passed output buffer to avoid copying data thereafter. + if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) { + found = true; + adjustedDirectBufferOutputsParameter[j] = + resetBuffer(outputs[outputIdx++]); + } + } + if (!found) { + throw new HadoopIllegalArgumentException( + "Inputs not fully corresponding to erasedIndexes in null places"); + } + } + // Use shared buffers for other positions (not set yet) + for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) { + if (adjustedDirectBufferOutputsParameter[i] == null) { + ByteBuffer buffer = checkGetDirectBuffer(bufferIdx, dataLen); + buffer.position(0); + buffer.limit(dataLen); + adjustedDirectBufferOutputsParameter[i] = resetBuffer(buffer); + bufferIdx++; + } + } + + doDecodeImpl(inputs, erasedOrNotToReadIndexes, + adjustedDirectBufferOutputsParameter); + } + + private byte[] checkGetBytesArrayBuffer(int idx, int bufferLen) { + if (bytesArrayBuffers[idx] == null || + bytesArrayBuffers[idx].length < bufferLen) { + bytesArrayBuffers[idx] = new byte[bufferLen]; + } + return bytesArrayBuffers[idx]; + } + + private ByteBuffer checkGetDirectBuffer(int idx, int bufferLen) { + if (directBuffers[idx] == null || + directBuffers[idx].capacity() < bufferLen) { + directBuffers[idx] = ByteBuffer.allocateDirect(bufferLen); + } + return directBuffers[idx]; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java index 1807da729a20a..ad7f32d9177e3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java @@ -32,6 +32,22 @@ public interface RawErasureDecoder extends RawErasureCoder { /** * Decode with inputs and erasedIndexes, generates outputs. + * How to prepare for inputs: + * 1. Create an array containing parity units + data units; + * 2. Set null in the array locations specified via erasedIndexes to indicate + * they're erased and no data are to read from; + * 3. Set null in the array locations for extra redundant items, as they're + * not necessary to read when decoding. For example in RS-6-3, if only 1 + * unit is really erased, then we have 2 extra items as redundant. They can + * be set as null to indicate no data will be used from them. + * + * For an example using RS (6, 3), assuming sources (d0, d1, d2, d3, d4, d5) + * and parities (p0, p1, p2), d2 being erased. We can and may want to use only + * 6 units like (d1, d3, d4, d5, p0, p2) to recover d2. We will have: + * inputs = [p0, null(p1), p2, null(d0), d1, null(d2), d3, d4, d5] + * erasedIndexes = [5] // index of d2 into inputs array + * outputs = [a-writable-buffer] + * * @param inputs inputs to read data from * @param erasedIndexes indexes of erased units in the inputs array * @param outputs outputs to write into for data generated according to @@ -41,7 +57,7 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs); /** - * Decode with inputs and erasedIndexes, generates outputs. + * Decode with inputs and erasedIndexes, generates outputs. More see above. * @param inputs inputs to read data from * @param erasedIndexes indexes of erased units in the inputs array * @param outputs outputs to write into for data generated according to @@ -50,7 +66,7 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs); /** - * Decode with inputs and erasedIndexes, generates outputs. + * Decode with inputs and erasedIndexes, generates outputs. More see above. * @param inputs inputs to read data from * @param erasedIndexes indexes of erased units in the inputs array * @param outputs outputs to write into for data generated according to diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java index a09105c4ad58c..e20e543422cad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java @@ -36,7 +36,7 @@ public XORRawDecoder(int numDataUnits, int numParityUnits) { protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) { ByteBuffer output = outputs[0]; - resetOutputBuffer(output); + resetBuffer(output); int erasedIdx = erasedIndexes[0]; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java index 894f20c7607e8..f4d242eefb026 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java @@ -34,7 +34,7 @@ public XORRawEncoder(int numDataUnits, int numParityUnits) { protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { ByteBuffer output = outputs[0]; - resetOutputBuffer(output); + resetBuffer(output); // Get the first buffer's data. int iIdx, oIdx; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java index 62b22c9fce309..03683b03deffc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java @@ -423,7 +423,7 @@ public void substitute(byte[][] p, int[] offsets, byte[] pi = p[i]; for (iIdx = offsets[i], oIdx = offset; iIdx < offsets[i] + len; iIdx++, oIdx++) { - int pij = pi[iIdx] & 0x000000FF; + int pij = pi != null ? pi[iIdx] & 0x000000FF : 0; q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]); } y = mulTable[x][y]; @@ -438,13 +438,15 @@ public void substitute(byte[][] p, int[] offsets, * @param q store the return result * @param x input field */ - public void substitute(ByteBuffer[] p, ByteBuffer q, int x) { + public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) { int y = 1, iIdx, oIdx; for (int i = 0; i < p.length; i++) { ByteBuffer pi = p[i]; - for (iIdx = pi.position(), oIdx = q.position(); - iIdx < pi.limit(); iIdx++, oIdx++) { - int pij = pi.get(iIdx) & 0x000000FF; + int pos = pi != null ? pi.position() : 0; + int limit = pi != null ? pi.limit() : len; + for (oIdx = q.position(), iIdx = pos; + iIdx < limit; iIdx++, oIdx++) { + int pij = pi != null ? pi.get(iIdx) & 0x000000FF : 0; q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y])); } y = mulTable[x][y]; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 3686695fb6d3c..9f50f335007a9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -35,7 +35,7 @@ public abstract class TestCoderBase { private Configuration conf; protected int numDataUnits; protected int numParityUnits; - protected int baseChunkSize = 16 * 1024; + protected int baseChunkSize = 513; private int chunkSize = baseChunkSize; private byte[] zeroChunkBytes; @@ -186,8 +186,9 @@ protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks, } /** - * Erase chunks to test the recovering of them. Before erasure clone them - * first so could return them. + * Erase some data chunks to test the recovering of them. As they're erased, + * we don't need to read them and will not have the buffers at all, so just + * set them as null. * @param dataChunks * @param parityChunks * @return clone of erased chunks @@ -198,50 +199,30 @@ protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks, erasedDataIndexes.length]; int idx = 0; - ECChunk chunk; for (int i = 0; i < erasedParityIndexes.length; i++) { - chunk = parityChunks[erasedParityIndexes[i]]; - toEraseChunks[idx ++] = cloneChunkWithData(chunk); - eraseDataFromChunk(chunk); + toEraseChunks[idx ++] = parityChunks[erasedParityIndexes[i]]; + parityChunks[erasedParityIndexes[i]] = null; } for (int i = 0; i < erasedDataIndexes.length; i++) { - chunk = dataChunks[erasedDataIndexes[i]]; - toEraseChunks[idx ++] = cloneChunkWithData(chunk); - eraseDataFromChunk(chunk); + toEraseChunks[idx ++] = dataChunks[erasedDataIndexes[i]]; + dataChunks[erasedDataIndexes[i]] = null; } return toEraseChunks; } /** - * Erase data from the specified chunks, putting ZERO bytes to the buffers. + * Erase data from the specified chunks, just setting them as null. * @param chunks */ protected void eraseDataFromChunks(ECChunk[] chunks) { for (int i = 0; i < chunks.length; i++) { - eraseDataFromChunk(chunks[i]); + chunks[i] = null; } } - /** - * Erase data from the specified chunk, putting ZERO bytes to the buffer. - * @param chunk with a buffer ready to read at the current position - */ - protected void eraseDataFromChunk(ECChunk chunk) { - ByteBuffer chunkBuffer = chunk.getBuffer(); - // Erase the data at the position, and restore the buffer ready for reading - // same many bytes but all ZERO. - int pos = chunkBuffer.position(); - int len = chunkBuffer.remaining(); - chunkBuffer.put(zeroChunkBytes, 0, len); - // Back to readable again after data erased - chunkBuffer.flip(); - chunkBuffer.position(pos); - chunkBuffer.limit(pos + len); - } - /** * Clone chunks along with copying the associated data. It respects how the * chunk buffer is allocated, direct or non-direct. It avoids affecting the diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index f9666b6b4b941..98fa95614c9d9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -232,7 +232,6 @@ protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks, TestBlock[] parityBlocks) { TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length + erasedParityIndexes.length]; - int idx = 0; TestBlock block; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 80ec04d0d4c4c..7b7ea42ca9527 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -32,89 +32,86 @@ public void setup() { } @Test - public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() { - prepare(null, 10, 4, new int[] {0}, new int[] {0}); - /** - * Doing twice to test if the coders can be repeatedly reused. This matters - * as the underlying coding buffers are shared, which may have bugs. - */ - testCoding(false); - testCoding(false); + public void testCoding_6x3_erasing_all_d() { + prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_10x4_erasing_p1() { - prepare(null, 10, 4, new int[0], new int[] {1}); - testCoding(true); - testCoding(true); + public void testCoding_6x3_erasing_d0_d2() { + prepare(null, 6, 3, new int[] {0, 2}, new int[]{}); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_10x4_erasing_d2() { - prepare(null, 10, 4, new int[] {2}, new int[] {}); - testCoding(true); - testCoding(true); + public void testCoding_6x3_erasing_d0() { + prepare(null, 6, 3, new int[]{0}, new int[0]); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_10x4_erasing_d0_p0() { - prepare(null, 10, 4, new int[] {0}, new int[] {0}); - testCoding(true); - testCoding(true); + public void testCoding_6x3_erasing_d2() { + prepare(null, 6, 3, new int[]{2}, new int[]{}); + testCodingDoMixAndTwice(); } @Test - public void testCodingBothBuffers_10x4_erasing_d0_p0() { - prepare(null, 10, 4, new int[] {0}, new int[] {0}); + public void testCoding_6x3_erasing_d0_p0() { + prepare(null, 6, 3, new int[]{0}, new int[]{0}); + testCodingDoMixAndTwice(); + } - /** - * Doing in mixed buffer usage model to test if the coders can be repeatedly - * reused with different buffer usage model. This matters as the underlying - * coding buffers are shared, which may have bugs. - */ - testCoding(true); - testCoding(false); - testCoding(true); - testCoding(false); + @Test + public void testCoding_6x3_erasing_all_p() { + prepare(null, 6, 3, new int[0], new int[]{0, 1, 2}); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() { - prepare(null, 10, 4, new int[]{2, 4}, new int[]{0}); - testCoding(true); + public void testCoding_6x3_erasing_p0() { + prepare(null, 6, 3, new int[0], new int[]{0}); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_usingFixedData_10x4_erasure_of_d2_d4_p0() { - prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}, true); - testCoding(true); + public void testCoding_6x3_erasing_p2() { + prepare(null, 6, 3, new int[0], new int[]{2}); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() { - prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1}); - testCoding(true); + public void testCoding_6x3_erasure_p0_p2() { + prepare(null, 6, 3, new int[0], new int[]{0, 2}); + testCodingDoMixAndTwice(); } @Test - public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() { - prepare(null, 3, 3, new int[] {0}, new int[] {0}); - testCoding(false); + public void testCoding_6x3_erasing_d0_p0_p1() { + prepare(null, 6, 3, new int[]{0}, new int[]{0, 1}); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_3x3_erasing_d0_p0() { - prepare(null, 3, 3, new int[] {0}, new int[] {0}); - testCoding(true); + public void testCoding_6x3_erasing_d0_d2_p2() { + prepare(null, 6, 3, new int[]{0, 2}, new int[]{2}); + testCodingDoMixAndTwice(); } @Test - public void testCodingNegative_10x4_erasing_d2_d4() { - prepare(null, 10, 4, new int[]{2, 4}, new int[0]); + public void testCodingNegative_6x3_erasing_d2_d4() { + prepare(null, 6, 3, new int[]{2, 4}, new int[0]); + testCodingDoMixAndTwice(); + } - testCodingWithBadInput(true); - testCodingWithBadOutput(false); - testCodingWithBadInput(true); - testCodingWithBadOutput(false); + @Test + public void testCodingNegative_6x3_erasing_too_many() { + prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1}); + testCodingWithErasingTooMany(); + } + + @Test + public void testCoding_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); + testCodingDoMixAndTwice(); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index cfaa2c542a523..dd5452b7a6eb8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -20,6 +20,7 @@ import org.apache.hadoop.io.erasurecode.ECChunk; import org.apache.hadoop.io.erasurecode.TestCoderBase; import org.junit.Assert; +import org.junit.Test; import java.lang.reflect.Constructor; @@ -32,6 +33,25 @@ public abstract class TestRawCoderBase extends TestCoderBase { private RawErasureEncoder encoder; private RawErasureDecoder decoder; + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + protected void testCodingDoMixAndTwice() { + testCodingDoMixed(); + testCodingDoMixed(); + } + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ + protected void testCodingDoMixed() { + testCoding(true); + testCoding(false); + } + /** * Generating source data, encoding, recovering and then verifying. * RawErasureCoder mainly uses ECChunk to pass input and output data buffers, @@ -85,6 +105,23 @@ protected void testCodingWithBadOutput(boolean usingDirectBuffer) { } } + @Test + public void testCodingWithErasingTooMany() { + try { + testCoding(true); + Assert.fail("Decoding test erasing too many should fail"); + } catch (Exception e) { + // Expected + } + + try { + testCoding(false); + Assert.fail("Decoding test erasing too many should fail"); + } catch (Exception e) { + // Expected + } + } + private void performTestCoding(int chunkSize, boolean useBadInput, boolean useBadOutput) { setChunkSize(chunkSize); @@ -110,6 +147,9 @@ private void performTestCoding(int chunkSize, ECChunk[] inputChunks = prepareInputChunksForDecoding( clonedDataChunks, parityChunks); + // Remove unnecessary chunks, allowing only least required chunks to be read. + ensureOnlyLeastRequiredChunks(inputChunks); + ECChunk[] recoveredChunks = prepareOutputChunksForDecoding(); if (useBadOutput) { corruptSomeChunk(recoveredChunks); @@ -131,6 +171,20 @@ private void prepareCoders() { } } + private void ensureOnlyLeastRequiredChunks(ECChunk[] inputChunks) { + int leastRequiredNum = numDataUnits; + int erasedNum = erasedDataIndexes.length + erasedParityIndexes.length; + int goodNum = inputChunks.length - erasedNum; + int redundantNum = goodNum - leastRequiredNum; + + for (int i = 0; i < inputChunks.length && redundantNum > 0; i++) { + if (inputChunks[i] != null) { + inputChunks[i] = null; // Setting it null, not needing it actually + redundantNum--; + } + } + } + /** * Create the raw erasure encoder to test * @return diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java index 327174ef836cc..48463ad1fd347 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java @@ -29,58 +29,35 @@ public class TestXORRawCoder extends TestRawCoderBase { public void setup() { this.encoderClass = XORRawEncoder.class; this.decoderClass = XORRawDecoder.class; - - this.numDataUnits = 10; - this.numParityUnits = 1; } @Test - public void testCodingNoDirectBuffer_erasing_d0() { + public void testCoding_10x1_erasing_d0() { prepare(null, 10, 1, new int[] {0}, new int[0]); - - /** - * Doing twice to test if the coders can be repeatedly reused. This matters - * as the underlying coding buffers are shared, which may have bugs. - */ - testCoding(false); - testCoding(false); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_erasing_p0() { + public void testCoding_10x1_erasing_p0() { prepare(null, 10, 1, new int[0], new int[] {0}); - - testCoding(true); - testCoding(true); + testCodingDoMixAndTwice(); } @Test - public void testCodingDirectBuffer_erasing_d0() { - prepare(null, 10, 1, new int[] {0}, new int[0]); - - testCoding(true); - testCoding(true); + public void testCoding_10x1_erasing_d5() { + prepare(null, 10, 1, new int[]{5}, new int[0]); + testCodingDoMixAndTwice(); } @Test - public void testCodingBothBuffers_erasing_d5() { - prepare(null, 10, 1, new int[]{5}, new int[0]); - - /** - * Doing in mixed buffer usage model to test if the coders can be repeatedly - * reused with different buffer usage model. This matters as the underlying - * coding buffers are shared, which may have bugs. - */ - testCoding(true); - testCoding(false); - testCoding(true); - testCoding(false); + public void testCodingNegative_10x1_erasing_too_many() { + prepare(null, 10, 1, new int[]{2}, new int[]{0}); + testCodingWithErasingTooMany(); } @Test - public void testCodingNegative_erasing_d5() { + public void testCodingNegative_10x1_erasing_d5() { prepare(null, 10, 1, new int[]{5}, new int[0]); - testCodingWithBadInput(true); testCodingWithBadOutput(false); testCodingWithBadInput(true); From c9e0268216584f1df1a7c6cd25d2cfb2bc6d1d3c Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 26 May 2015 12:32:16 -0700 Subject: [PATCH 140/212] Addendum fix for HDFS-7912. --- .../server/blockmanagement/BlockManager.java | 2 +- .../hadoop/hdfs/TestRecoverStripedFile.java | 3 ++- .../TestPendingReplication.java | 5 +++-- .../blockmanagement/TestReplicationPolicy.java | 18 ++++-------------- 4 files changed, 10 insertions(+), 18 deletions(-) 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 8f1f6b71c951e..32757f925bd76 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 @@ -1819,7 +1819,7 @@ private void processPendingReplications() { * Use the blockinfo from the blocksmap to be certain we're working * with the most up-to-date block information (e.g. genstamp). */ - BlockInfoContiguous bi = blocksMap.getStoredBlock(timedOutItems[i]); + BlockInfo bi = blocksMap.getStoredBlock(timedOutItems[i]); if (bi == null) { continue; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index e5e324cfbbbb6..9e44761d20688 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -50,6 +50,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; public class TestRecoverStripedFile { public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class); @@ -340,7 +341,7 @@ private LocatedBlocks getLocatedBlocks(Path file) throws IOException { private void testCreateStripedFile(Path file, int dataLen) throws IOException { final byte[] data = new byte[dataLen]; - DFSUtil.getRandom().nextBytes(data); + ThreadLocalRandom.current().nextBytes(data); writeContents(file, data); } 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 b714afa6840cc..c00037b071864 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 @@ -186,7 +186,7 @@ public void testProcessPendingReplications() throws Exception { block = new Block(1, 1, 0); blockInfo = new BlockInfoContiguous(block, (short) 3); - pendingReplications.increment(block, + pendingReplications.increment(blockInfo, DatanodeStorageInfo.toDatanodeDescriptors( DFSTestUtil.createDatanodeStorageInfos(1))); BlockCollection bc = Mockito.mock(BlockCollection.class); @@ -201,7 +201,8 @@ public void testProcessPendingReplications() throws Exception { // Add a second block to pendingReplications that has no // corresponding entry in blocksmap block = new Block(2, 2, 0); - pendingReplications.increment(block, + blockInfo = new BlockInfoContiguous(block, (short) 3); + pendingReplications.increment(blockInfo, DatanodeStorageInfo.toDatanodeDescriptors( DFSTestUtil.createDatanodeStorageInfos(1))); 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 1916b3cd3fcc5..6e92264aa5b8b 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 @@ -1209,13 +1209,8 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; -<<<<<<< HEAD - Block block1 = new Block(ThreadLocalRandom.current().nextLong()); - Block block2 = new Block(ThreadLocalRandom.current().nextLong()); -======= - BlockInfo block1 = genBlockInfo(random.nextLong()); - BlockInfo block2 = genBlockInfo(random.nextLong()); ->>>>>>> 3e6f458... HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao. + BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); @@ -1277,13 +1272,8 @@ public void testupdateNeededReplicationsDoesNotCauseSkippedReplication() BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; -<<<<<<< HEAD - Block block1 = new Block(ThreadLocalRandom.current().nextLong()); - Block block2 = new Block(ThreadLocalRandom.current().nextLong()); -======= - BlockInfo block1 = genBlockInfo(random.nextLong()); - BlockInfo block2 = genBlockInfo(random.nextLong()); ->>>>>>> 3e6f458... HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao. + BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); From 1299357a05c52ad45513ed0ea854edc9c7ec3de8 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 26 May 2015 16:06:50 -0700 Subject: [PATCH 141/212] HDFS-8479. Erasure coding: fix striping related logic in FSDirWriteFileOp to sync with HDFS-8421. Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hdfs/server/namenode/FSDirWriteFileOp.java | 7 +++++++ .../hadoop/hdfs/server/namenode/FSDirectory.java | 5 ----- .../blockmanagement/TestReplicationPolicy.java | 14 +++++++++++--- 4 files changed, 21 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index d045ee5fb2187..ccc3cb0e27340 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -259,3 +259,6 @@ HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng) HDFS-8408. Revisit and refactor ErasureCodingInfo (vinayakumarb) + + HDFS-8479. Erasure coding: fix striping related logic in FSDirWriteFileOp to + sync with HDFS-8421. (Zhe Zhang via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 11b6d8f2f4f0a..c037cd74a98c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -494,6 +494,10 @@ static INodeFile addFileForEditLog( try { INodesInPath iip = fsd.addINode(existing, newNode); if (iip != null) { + // check if the file is in an EC zone + if (fsd.isInECZone(iip)) { + newNode.addStripedBlocksFeature(); + } if (aclEntries != null) { AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID); } @@ -582,6 +586,9 @@ private static INodesInPath addFile( fsd.writeLock(); try { newiip = fsd.addINode(existing, newNode); + if (newiip != null && fsd.isInECZone(newiip)) { + newNode.addStripedBlocksFeature(); + } } finally { fsd.writeUnlock(); } 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 0e3694f31b1ca..32c8e056af48c 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 @@ -33,10 +33,8 @@ import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.XAttrHelper; @@ -54,9 +52,6 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; 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 6e92264aa5b8b..e699e7f14d6dc 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.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; @@ -1209,8 +1208,17 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; - BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); - BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + long blkID1 = ThreadLocalRandom.current().nextLong(); + if (blkID1 < 0) { + blkID1 *= -1; + } + long blkID2 = ThreadLocalRandom.current().nextLong(); + if (blkID2 < 0) { + blkID2 *= -1; + } + + BlockInfo block1 = genBlockInfo(blkID1); + BlockInfo block2 = genBlockInfo(blkID2); // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); From 014bd32c58ccd7b4ff66ff1180f7fe9ae25484ef Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 1 Jun 2015 09:37:29 -0700 Subject: [PATCH 142/212] HDFS-8481. Erasure coding: remove workarounds in client side stripped blocks recovering. Contributed by Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 11 +++- .../hadoop/hdfs/util/StripedBlockUtil.java | 62 ++++++++++++------- .../hdfs/TestDFSStripedInputStream.java | 12 ++-- .../hadoop/hdfs/TestWriteReadStripedFile.java | 8 +-- 5 files changed, 60 insertions(+), 36 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index ccc3cb0e27340..01d8cf032a47b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -262,3 +262,6 @@ HDFS-8479. Erasure coding: fix striping related logic in FSDirWriteFileOp to sync with HDFS-8421. (Zhe Zhang via jing9) + + HDFS-8481. Erasure coding: remove workarounds in client side stripped blocks + recovering. (zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 3b7eb5881402b..b4aa033545749 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -31,6 +31,7 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.finalizeDecodeInputs; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getStartOffsetsForInternalBlocks; @@ -41,6 +42,8 @@ import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.net.NetUtils; import org.apache.htrace.Span; import org.apache.htrace.Trace; @@ -117,6 +120,8 @@ boolean include(long pos) { /** the buffer for a complete stripe */ private ByteBuffer curStripeBuf; private final ECSchema schema; + private final RawErasureDecoder decoder; + /** * indicate the start/end offset of the current buffered stripe in the * block group @@ -139,6 +144,7 @@ boolean include(long pos) { curStripeRange = new StripeRange(0, 0); readingService = new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); + decoder = new RSRawDecoder(dataBlkNum, parityBlkNum); if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Creating an striped input stream for file " + src); } @@ -591,8 +597,9 @@ private void fetchOneStripe(LocatedStripedBlock blockGroup, } if (alignedStripe.missingChunksNum > 0) { - decodeAndFillBuffer(decodeInputs, buf, alignedStripe, - dataBlkNum, parityBlkNum); + finalizeDecodeInputs(decodeInputs, alignedStripe); + decodeAndFillBuffer(decodeInputs, buf, alignedStripe, parityBlkNum, + decoder); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 8f63236aa44f9..e75209fcc2244 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -32,6 +32,7 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import java.util.*; import java.io.IOException; @@ -246,19 +247,36 @@ public static long spaceConsumedByStripedBlock(long numDataBlkBytes, /** * Initialize the decoding input buffers based on the chunk states in an - * AlignedStripe + * {@link AlignedStripe}. For each chunk that was not initially requested, + * schedule a new fetch request with the decoding input buffer as transfer + * destination. */ public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum) { byte[][] decodeInputs = new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()]; for (int i = 0; i < alignedStripe.chunks.length; i++) { - StripingChunk chunk = alignedStripe.chunks[i]; - if (chunk == null) { + if (alignedStripe.chunks[i] == null) { alignedStripe.chunks[i] = new StripingChunk(decodeInputs[i]); alignedStripe.chunks[i].offsetsInBuf.add(0); alignedStripe.chunks[i].lengthsInBuf.add((int) alignedStripe.getSpanInBlock()); - } else if (chunk.state == StripingChunk.FETCHED) { + } + } + return decodeInputs; + } + + /** + * Some fetched {@link StripingChunk} might be stored in original application + * buffer instead of prepared decode input buffers. Some others are beyond + * the range of the internal blocks and should correspond to all zero bytes. + * When all pending requests have returned, this method should be called to + * finalize decode input buffers. + */ + public static void finalizeDecodeInputs(final byte[][] decodeInputs, + AlignedStripe alignedStripe) { + for (int i = 0; i < alignedStripe.chunks.length; i++) { + StripingChunk chunk = alignedStripe.chunks[i]; + if (chunk.state == StripingChunk.FETCHED) { int posInBuf = 0; for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { System.arraycopy(chunk.buf, chunk.offsetsInBuf.get(j), @@ -267,39 +285,41 @@ public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, } } else if (chunk.state == StripingChunk.ALLZERO) { Arrays.fill(decodeInputs[i], (byte)0); + } else { + decodeInputs[i] = null; } } - return decodeInputs; } - /** - * Decode based on the given input buffers and schema + * Decode based on the given input buffers and schema. */ - public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf, - AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum) { + public static void decodeAndFillBuffer(final byte[][] decodeInputs, + byte[] buf, AlignedStripe alignedStripe, int parityBlkNum, + RawErasureDecoder decoder) { + // Step 1: prepare indices and output buffers for missing data units int[] decodeIndices = new int[parityBlkNum]; int pos = 0; for (int i = 0; i < alignedStripe.chunks.length; i++) { - if (alignedStripe.chunks[i].state != StripingChunk.FETCHED && - alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { + if (alignedStripe.chunks[i].state == StripingChunk.MISSING){ decodeIndices[pos++] = i; } } + decodeIndices = Arrays.copyOf(decodeIndices, pos); + byte[][] decodeOutputs = + new byte[decodeIndices.length][(int) alignedStripe.getSpanInBlock()]; - byte[][] outputs = new byte[parityBlkNum][(int) alignedStripe.getSpanInBlock()]; - RSRawDecoder rsRawDecoder = new RSRawDecoder(dataBlkNum, parityBlkNum); - rsRawDecoder.decode(decodeInputs, decodeIndices, outputs); + // Step 2: decode into prepared output buffers + decoder.decode(decodeInputs, decodeIndices, decodeOutputs); - for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { - StripingChunk chunk = alignedStripe.chunks[i]; + // Step 3: fill original application buffer with decoded data + for (int i = 0; i < decodeIndices.length; i++) { + int missingBlkIdx = decodeIndices[i]; + StripingChunk chunk = alignedStripe.chunks[missingBlkIdx]; if (chunk.state == StripingChunk.MISSING) { int srcPos = 0; for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { - //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 -// System.arraycopy(outputs[i], srcPos, buf, chunk.offsetsInBuf.get(j), -// chunk.lengthsInBuf.get(j)); - Arrays.fill(buf, chunk.offsetsInBuf.get(j), - chunk.offsetsInBuf.get(j) + chunk.lengthsInBuf.get(j), (byte)7); + System.arraycopy(decodeOutputs[i], srcPos, buf, chunk.offsetsInBuf.get(j), + chunk.lengthsInBuf.get(j)); srcPos += chunk.lengthsInBuf.get(j); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index a71441f021d20..ce563254ae95d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -221,13 +221,13 @@ public void testPreadWithDNFailure() throws Exception { decodeInputs[DATA_BLK_NUM][k] = SimulatedFSDataset.simulatedByte( new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk); } -// RSRawDecoder rsRawDecoder = new RSRawDecoder(); -// rsRawDecoder.initialize(DATA_BLK_NUM, PARITY_BLK_NUM, CELLSIZE); -// rsRawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs); + for (int m : missingBlkIdx) { + decodeInputs[m] = null; + } + RSRawDecoder rsRawDecoder = new RSRawDecoder(DATA_BLK_NUM, PARITY_BLK_NUM); + rsRawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs); int posInBuf = i * CELLSIZE * DATA_BLK_NUM + failedDNIdx * CELLSIZE; -// System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE); - //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 - Arrays.fill(expected, posInBuf, posInBuf + CELLSIZE, (byte)7); + System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE); } int delta = 10; int done = 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 70802fb59a826..b0436a68c56fb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -382,15 +382,9 @@ public void testWritePreadWithDNFailure() throws IOException { Assert.assertEquals("The length of file should be the same to write size", length - startOffsetInFile, readLen); - RSRawDecoder rsRawDecoder = new RSRawDecoder(dataBlocks, parityBlocks); byte[] expected = new byte[readLen]; for (int i = startOffsetInFile; i < length; i++) { - //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938 - if ((i / cellSize) % dataBlocks == failedDNIdx) { - expected[i - startOffsetInFile] = (byte)7; - } else { - expected[i - startOffsetInFile] = getByte(i); - } + expected[i - startOffsetInFile] = getByte(i); } for (int i = startOffsetInFile; i < length; i++) { Assert.assertEquals("Byte at " + i + " should be the same", From 0799e1e4b6a343d3960619b844e15037a460a3ef Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Tue, 2 Jun 2015 22:05:16 +0800 Subject: [PATCH 143/212] HADOOP-12011 Allow to dump verbose information to ease debugging in raw erasure coders. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 + .../erasurecode/rawcoder/util/DumpUtil.java | 85 +++++++++++++++++++ .../hadoop/io/erasurecode/TestCoderBase.java | 42 +++++++++ .../erasurecode/rawcoder/TestRSRawCoder.java | 1 + .../rawcoder/TestRawCoderBase.java | 6 ++ 5 files changed, 137 insertions(+) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 0c244737849d1..3559436a0760f 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -61,4 +61,7 @@ (vinayakumarb) HADOOP-11847. Enhance raw coder allowing to read least required inputs in decoding. + (Kai Zheng) + + HADOOP-12011. Allow to dump verbose information to ease debugging in raw erasure coders (Kai Zheng) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java new file mode 100644 index 0000000000000..c8f133f765585 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java @@ -0,0 +1,85 @@ +/** + * 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.io.erasurecode.rawcoder.util; + +import org.apache.hadoop.io.erasurecode.ECChunk; + +/** + * A dump utility class for debugging data erasure coding/decoding issues. Don't + * suggest they are used in runtime production codes. + */ +public final class DumpUtil { + private static final String HEX_CHARS_STR = "0123456789ABCDEF"; + private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray(); + + private DumpUtil() { + // No called + } + + /** + * Convert bytes into format like 0x02 02 00 80. + */ + public static String bytesToHex(byte[] bytes, int limit) { + if (limit > bytes.length) { + limit = bytes.length; + } + int len = limit * 2; + len += limit; // for ' ' appended for each char + len += 2; // for '0x' prefix + char[] hexChars = new char[len]; + hexChars[0] = '0'; + hexChars[1] = 'x'; + for (int j = 0; j < limit; j++) { + int v = bytes[j] & 0xFF; + hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4]; + hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F]; + hexChars[j * 3 + 4] = ' '; + } + + return new String(hexChars); + } + + /** + * Print data in hex format in an array of chunks. + * @param header + * @param chunks + */ + public static void dumpChunks(String header, ECChunk[] chunks) { + System.out.println(); + System.out.println(header); + for (int i = 0; i < chunks.length; i++) { + dumpChunk(chunks[i]); + } + System.out.println(); + } + + /** + * Print data in hex format in a chunk. + * @param chunk + */ + public static void dumpChunk(ECChunk chunk) { + String str; + if (chunk == null) { + str = ""; + } else { + byte[] bytes = chunk.toBytesArray(); + str = DumpUtil.bytesToHex(bytes, 16); + } + System.out.println(str); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 9f50f335007a9..10edae8d72f33 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -18,6 +18,7 @@ package org.apache.hadoop.io.erasurecode; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil; import java.nio.ByteBuffer; import java.util.Arrays; @@ -32,6 +33,8 @@ public abstract class TestCoderBase { protected static Random RAND = new Random(); + private boolean allowDump = true; + private Configuration conf; protected int numDataUnits; protected int numParityUnits; @@ -67,6 +70,15 @@ protected void setChunkSize(int chunkSize) { this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default } + /** + * Set true during setup if want to dump test settings and coding data, + * useful in debugging. + * @param allowDump + */ + protected void setAllowDump(boolean allowDump) { + this.allowDump = allowDump; + } + /** * Prepare before running the case. * @param conf @@ -430,6 +442,36 @@ protected byte[][] toArrays(ECChunk[] chunks) { return bytesArr; } + /** + * Dump all the settings used in the test case if allowDump is enabled. + */ + protected void dumpSetting() { + if (allowDump) { + StringBuilder sb = new StringBuilder("Erasure coder test settings:\n"); + sb.append(" numDataUnits=").append(numDataUnits); + sb.append(" numParityUnits=").append(numParityUnits); + sb.append(" chunkSize=").append(chunkSize).append("\n"); + + sb.append(" erasedDataIndexes="). + append(Arrays.toString(erasedDataIndexes)); + sb.append(" erasedParityIndexes="). + append(Arrays.toString(erasedParityIndexes)); + sb.append(" usingDirectBuffer=").append(usingDirectBuffer).append("\n"); + + System.out.println(sb.toString()); + } + } + + /** + * Dump chunks prefixed with a header if allowDump is enabled. + * @param header + * @param chunks + */ + protected void dumpChunks(String header, ECChunk[] chunks) { + if (allowDump) { + DumpUtil.dumpChunks(header, chunks); + } + } /** * Make some chunk messy or not correct any more diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java index 7b7ea42ca9527..a35a4dd5c3b86 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -29,6 +29,7 @@ public class TestRSRawCoder extends TestRSRawCoderBase { public void setup() { this.encoderClass = RSRawEncoder.class; this.decoderClass = RSRawDecoder.class; + setAllowDump(false); // Change to true to allow verbose dump for debugging } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index dd5452b7a6eb8..587ce96bf12d0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -126,11 +126,14 @@ private void performTestCoding(int chunkSize, boolean useBadInput, boolean useBadOutput) { setChunkSize(chunkSize); + dumpSetting(); + // Generate data and encode ECChunk[] dataChunks = prepareDataChunksForEncoding(); if (useBadInput) { corruptSomeChunk(dataChunks); } + dumpChunks("Testing data chunks", dataChunks); ECChunk[] parityChunks = prepareParityChunksForEncoding(); @@ -139,6 +142,7 @@ private void performTestCoding(int chunkSize, ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks); encoder.encode(dataChunks, parityChunks); + dumpChunks("Encoded parity chunks", parityChunks); // Backup and erase some chunks ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks, parityChunks); @@ -155,7 +159,9 @@ private void performTestCoding(int chunkSize, corruptSomeChunk(recoveredChunks); } + dumpChunks("Decoding input chunks", inputChunks); decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks); + dumpChunks("Decoded/recovered chunks", recoveredChunks); // Compare compareAndVerify(backupChunks, recoveredChunks); From 9b54e66f3ea6a5a6945d18ba534cfb4f2419facb Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 2 Jun 2015 16:38:43 +0530 Subject: [PATCH 144/212] HDFS-8336. Expose some administrative erasure coding operations to HdfsAdmin (Contributed by Uma Maheswara Rao G) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../apache/hadoop/hdfs/client/HdfsAdmin.java | 40 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 01d8cf032a47b..e7335b2165bb2 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -265,3 +265,6 @@ HDFS-8481. Erasure coding: remove workarounds in client side stripped blocks recovering. (zhz) + + HDFS-8336. Expose some administrative erasure coding operations to HdfsAdmin + (Uma Maheswara Rao G via vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java index 84499bb3d4d98..5a3c885b49bbd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java @@ -37,9 +37,11 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.hdfs.tools.DFSAdmin; +import org.apache.hadoop.io.erasurecode.ECSchema; /** * The public API for performing administrative functions on HDFS. Those writing @@ -363,4 +365,42 @@ public void setStoragePolicy(final Path src, final String policyName) throws IOException { dfs.setStoragePolicy(src, policyName); } + + /** + * Create the ErasureCoding zone + * + * @param path + * Directory to create the ErasureCoding zone + * @param schema + * ECSchema for the zone. If not specified default will be used. + * @param cellSize + * Cellsize for the striped ErasureCoding + * @throws IOException + */ + public void createErasureCodingZone(final Path path, final ECSchema schema, + final int cellSize) throws IOException { + dfs.createErasureCodingZone(path, schema, cellSize); + } + + /** + * Get the ErasureCoding zone information for the specified path + * + * @param path + * @return Returns the zone information if path is in EC zone, null otherwise + * @throws IOException + */ + public ErasureCodingZone getErasureCodingZone(final Path path) + throws IOException { + return dfs.getErasureCodingZone(path); + } + + /** + * Get the ErasureCoding schemas supported. + * + * @return ECSchemas + * @throws IOException + */ + public ECSchema[] getECSchemas() throws IOException { + return dfs.getClient().getECSchemas(); + } } From 2d847e7d620f97ebe8e9297361e2541d7d0ecd50 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 2 Jun 2015 16:41:19 +0530 Subject: [PATCH 145/212] HDFS-8444. Erasure Coding: fix cannot rename a zone dir (Contributed by Walter Su) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../namenode/ErasureCodingZoneManager.java | 9 ++++++-- .../hadoop/hdfs/TestErasureCodingZones.java | 21 ++++++++++++------- 3 files changed, 23 insertions(+), 10 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index e7335b2165bb2..c3c55c731abf6 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -268,3 +268,6 @@ HDFS-8336. Expose some administrative erasure coding operations to HdfsAdmin (Uma Maheswara Rao G via vinayakumarb) + + HDFS-8444. Erasure Coding: fix cannot rename a zone dir + (Walter Su via vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 4b3e747927b87..e853829be56ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -153,8 +153,13 @@ XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize) void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src) throws IOException { assert dir.hasReadLock(); - final ECSchema srcSchema = getECSchema(srcIIP); - final ECSchema dstSchema = getECSchema(dstIIP); + final ErasureCodingZone srcZone = getECZone(srcIIP); + final ErasureCodingZone dstZone = getECZone(dstIIP); + if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) { + return; + } + final ECSchema srcSchema = (srcZone != null) ? srcZone.getSchema() : null; + final ECSchema dstSchema = (dstZone != null) ? dstZone.getSchema() : null; if ((srcSchema != null && !srcSchema.equals(dstSchema)) || (dstSchema != null && !dstSchema.equals(srcSchema))) { throw new IOException( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index d724b53d6fbf7..1a10ebff9b8ca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -119,15 +119,20 @@ public void testMoveValidity() throws IOException, InterruptedException { final Path srcFile = new Path(srcECDir, "foo"); fs.create(srcFile); - /* Verify that a file can be moved between 2 EC zones */ - try { - fs.rename(srcFile, dstECDir); - } catch (IOException e) { - fail("A file should be able to move between 2 EC zones " + e); - } + // Test move dir + // Move EC dir under non-EC dir + final Path newDir = new Path("/srcEC_new"); + fs.rename(srcECDir, newDir); + fs.rename(newDir, srcECDir); // move back + + // Move EC dir under another EC dir + fs.rename(srcECDir, dstECDir); + fs.rename(new Path("/dstEC/srcEC"), srcECDir); // move back - // Move the file back - fs.rename(new Path(dstECDir, "foo"), srcECDir); + // Test move file + /* Verify that a file can be moved between 2 EC zones */ + fs.rename(srcFile, dstECDir); + fs.rename(new Path(dstECDir, "foo"), srcECDir); // move back /* Verify that a file cannot be moved from a non-EC dir to an EC zone */ final Path nonECDir = new Path("/nonEC"); From 71329e817b99dbee630f902fa3640c3c93f04a44 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 2 Jun 2015 15:35:49 -0700 Subject: [PATCH 146/212] HDFS-8517. Fix a decoding issue in stripped block recovering in client side. Contributed by Kai Zheng. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 7 +- .../hadoop/hdfs/util/StripedBlockUtil.java | 57 +++-- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 14 +- .../hadoop/hdfs/StripedFileTestUtil.java | 59 +++++ .../hdfs/TestDFSStripedInputStream.java | 7 +- .../hdfs/TestReadStripedFileWithDecoding.java | 108 ++++++++ .../hadoop/hdfs/TestWriteReadStripedFile.java | 238 ++++++------------ 8 files changed, 305 insertions(+), 188 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index c3c55c731abf6..fa0a8e2c543ba 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -271,3 +271,6 @@ HDFS-8444. Erasure Coding: fix cannot rename a zone dir (Walter Su via vinayakumarb) + + HDFS-8517. Fix a decoding issue in stripped block recovering in client side. + (Kai Zheng via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index b4aa033545749..228368ba4925f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -597,9 +597,10 @@ private void fetchOneStripe(LocatedStripedBlock blockGroup, } if (alignedStripe.missingChunksNum > 0) { - finalizeDecodeInputs(decodeInputs, alignedStripe); - decodeAndFillBuffer(decodeInputs, buf, alignedStripe, parityBlkNum, - decoder); + finalizeDecodeInputs(decodeInputs, dataBlkNum, parityBlkNum, + alignedStripe); + decodeAndFillBuffer(decodeInputs, buf, alignedStripe, dataBlkNum, + parityBlkNum, decoder); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index e75209fcc2244..80321ef9590ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -31,7 +31,6 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import java.util.*; @@ -257,7 +256,8 @@ public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()]; for (int i = 0; i < alignedStripe.chunks.length; i++) { if (alignedStripe.chunks[i] == null) { - alignedStripe.chunks[i] = new StripingChunk(decodeInputs[i]); + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); alignedStripe.chunks[i].offsetsInBuf.add(0); alignedStripe.chunks[i].lengthsInBuf.add((int) alignedStripe.getSpanInBlock()); } @@ -273,35 +273,57 @@ public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, * finalize decode input buffers. */ public static void finalizeDecodeInputs(final byte[][] decodeInputs, - AlignedStripe alignedStripe) { + int dataBlkNum, int parityBlkNum, AlignedStripe alignedStripe) { for (int i = 0; i < alignedStripe.chunks.length; i++) { - StripingChunk chunk = alignedStripe.chunks[i]; + final StripingChunk chunk = alignedStripe.chunks[i]; + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); if (chunk.state == StripingChunk.FETCHED) { int posInBuf = 0; for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { System.arraycopy(chunk.buf, chunk.offsetsInBuf.get(j), - decodeInputs[i], posInBuf, chunk.lengthsInBuf.get(j)); + decodeInputs[decodeIndex], posInBuf, chunk.lengthsInBuf.get(j)); posInBuf += chunk.lengthsInBuf.get(j); } } else if (chunk.state == StripingChunk.ALLZERO) { - Arrays.fill(decodeInputs[i], (byte)0); + Arrays.fill(decodeInputs[decodeIndex], (byte) 0); } else { - decodeInputs[i] = null; + decodeInputs[decodeIndex] = null; } } } + + /** + * Currently decoding requires parity chunks are before data chunks. + * The indices are opposite to what we store in NN. In future we may + * improve the decoding to make the indices order the same as in NN. + * + * @param index The index to convert + * @param dataBlkNum The number of data blocks + * @param parityBlkNum The number of parity blocks + * @return converted index + */ + public static int convertIndex4Decode(int index, int dataBlkNum, + int parityBlkNum) { + return index < dataBlkNum ? index + parityBlkNum : index - dataBlkNum; + } + + public static int convertDecodeIndexBack(int index, int dataBlkNum, + int parityBlkNum) { + return index < parityBlkNum ? index + dataBlkNum : index - parityBlkNum; + } + /** * Decode based on the given input buffers and schema. */ public static void decodeAndFillBuffer(final byte[][] decodeInputs, - byte[] buf, AlignedStripe alignedStripe, int parityBlkNum, + byte[] buf, AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum, RawErasureDecoder decoder) { // Step 1: prepare indices and output buffers for missing data units int[] decodeIndices = new int[parityBlkNum]; int pos = 0; for (int i = 0; i < alignedStripe.chunks.length; i++) { if (alignedStripe.chunks[i].state == StripingChunk.MISSING){ - decodeIndices[pos++] = i; + decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum); } } decodeIndices = Arrays.copyOf(decodeIndices, pos); @@ -313,13 +335,14 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, // Step 3: fill original application buffer with decoded data for (int i = 0; i < decodeIndices.length; i++) { - int missingBlkIdx = decodeIndices[i]; + int missingBlkIdx = convertDecodeIndexBack(decodeIndices[i], + dataBlkNum, parityBlkNum); StripingChunk chunk = alignedStripe.chunks[missingBlkIdx]; if (chunk.state == StripingChunk.MISSING) { int srcPos = 0; for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { - System.arraycopy(decodeOutputs[i], srcPos, buf, chunk.offsetsInBuf.get(j), - chunk.lengthsInBuf.get(j)); + System.arraycopy(decodeOutputs[i], srcPos, buf, + chunk.offsetsInBuf.get(j), chunk.lengthsInBuf.get(j)); srcPos += chunk.lengthsInBuf.get(j); } } @@ -330,7 +353,7 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, * This method divides a requested byte range into an array of inclusive * {@link AlignedStripe}. * @param ecSchema The codec schema for the file, which carries the numbers - * of data / parity blocks, as well as cell size + * of data / parity blocks * @param cellSize Cell size of stripe * @param blockGroup The striped block group * @param rangeStartInBlockGroup The byte range's start offset in block group @@ -345,7 +368,6 @@ public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf, int offsetInBuf) { - // TODO: change ECSchema naming to use cell size instead of chunk size // Step 0: analyze range and calculate basic parameters int dataBlkNum = ecSchema.getNumDataUnits(); @@ -362,8 +384,7 @@ public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); // Step 4: calculate each chunk's position in destination buffer - calcualteChunkPositionsInBuf(ecSchema, cellSize, stripes, cells, buf, - offsetInBuf); + calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf); // Step 5: prepare ALLZERO blocks prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum); @@ -508,8 +529,8 @@ private static AlignedStripe[] mergeRangesForInternalBlocks( return stripes.toArray(new AlignedStripe[stripes.size()]); } - private static void calcualteChunkPositionsInBuf(ECSchema ecSchema, - int cellSize, AlignedStripe[] stripes, StripingCell[] cells, byte[] buf, + private static void calcualteChunkPositionsInBuf(int cellSize, + AlignedStripe[] stripes, StripingCell[] cells, byte[] buf, int offsetInBuf) { /** * | <--------------- AlignedStripe --------------->| 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 82c078141fb2a..6cd7003b3ee9b 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 @@ -790,15 +790,21 @@ public static byte[] readFileAsBytes(File f) throws IOException { return os.toByteArray(); } - /* Write the given string to the given file */ - public static void writeFile(FileSystem fs, Path p, String s) + /* Write the given bytes to the given file */ + public static void writeFile(FileSystem fs, Path p, byte[] bytes) throws IOException { if (fs.exists(p)) { fs.delete(p, true); } - InputStream is = new ByteArrayInputStream(s.getBytes()); + InputStream is = new ByteArrayInputStream(bytes); FSDataOutputStream os = fs.create(p); - IOUtils.copyBytes(is, os, s.length(), true); + IOUtils.copyBytes(is, os, bytes.length, true); + } + + /* Write the given string to the given file */ + public static void writeFile(FileSystem fs, Path p, String s) + throws IOException { + writeFile(fs, p, s.getBytes()); } /* Append the given string to the given file */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java new file mode 100644 index 0000000000000..54367d743af63 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; + +import java.io.IOException; +import java.util.Random; + +public class StripedFileTestUtil { + static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + + static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + static final int stripesPerBlock = 4; + static final int blockSize = cellSize * stripesPerBlock; + static final int numDNs = dataBlocks + parityBlocks + 2; + + static final Random r = new Random(); + + static byte[] generateBytes(int cnt) { + byte[] bytes = new byte[cnt]; + for (int i = 0; i < cnt; i++) { + bytes[i] = getByte(i); + } + return bytes; + } + + static int readAll(FSDataInputStream in, byte[] buf) throws IOException { + int readLen = 0; + int ret; + while ((ret = in.read(buf, readLen, buf.length - readLen)) >= 0 && + readLen <= buf.length) { + readLen += ret; + } + return readLen; + } + + static byte getByte(long pos) { + final int mod = 29; + return (byte) (pos % mod + 1); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index ce563254ae95d..b64e690e1eb33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -208,17 +208,18 @@ public void testPreadWithDNFailure() throws Exception { // Update the expected content for decoded data for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { byte[][] decodeInputs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][CELLSIZE]; - int[] missingBlkIdx = new int[]{failedDNIdx, DATA_BLK_NUM+1, DATA_BLK_NUM+2}; + int[] missingBlkIdx = new int[]{failedDNIdx + PARITY_BLK_NUM, 1, 2}; byte[][] decodeOutputs = new byte[PARITY_BLK_NUM][CELLSIZE]; for (int j = 0; j < DATA_BLK_NUM; j++) { int posInBuf = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE; if (j != failedDNIdx) { - System.arraycopy(expected, posInBuf, decodeInputs[j], 0, CELLSIZE); + System.arraycopy(expected, posInBuf, decodeInputs[j + PARITY_BLK_NUM], + 0, CELLSIZE); } } for (int k = 0; k < CELLSIZE; k++) { int posInBlk = i * CELLSIZE + k; - decodeInputs[DATA_BLK_NUM][k] = SimulatedFSDataset.simulatedByte( + decodeInputs[0][k] = SimulatedFSDataset.simulatedByte( new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk); } for (int m : missingBlkIdx) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java new file mode 100644 index 0000000000000..7397caf470218 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -0,0 +1,108 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; + +public class TestReadStripedFileWithDecoding { + + private MiniDFSCluster cluster; + private FileSystem fs; + + @Before + public void setup() throws IOException { + Configuration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().createErasureCodingZone("/", + null, cellSize); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testWritePreadWithDNFailure1() throws IOException { + testWritePreadWithDNFailure("/foo", 0); + } + + @Test + public void testWritePreadWithDNFailure2() throws IOException { + testWritePreadWithDNFailure("/foo", cellSize * 5); + } + + private void testWritePreadWithDNFailure(String file, int startOffsetInFile) + throws IOException { + final int failedDNIdx = 2; + final int length = cellSize * (dataBlocks + 2); + Path testPath = new Path(file); + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, testPath, bytes); + + // shut down the DN that holds the last internal data block + BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, + cellSize); + String name = (locs[0].getNames())[failedDNIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + break; + } + } + + // pread + try (FSDataInputStream fsdis = fs.open(testPath)) { + byte[] buf = new byte[length]; + int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length); + Assert.assertEquals("The length of file should be the same to write size", + length - startOffsetInFile, readLen); + + byte[] expected = new byte[readLen]; + for (int i = startOffsetInFile; i < length; i++) { + expected[i - startOffsetInFile] = StripedFileTestUtil.getByte(i); + } + + for (int i = startOffsetInFile; i < length; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", + expected[i - startOffsetInFile], buf[i - startOffsetInFile]); + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index b0436a68c56fb..e2e52467a625c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -18,17 +18,13 @@ package org.apache.hadoop.hdfs; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.web.ByteRangeInputStream; import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -37,34 +33,30 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Random; -public class TestWriteReadStripedFile { - private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - - private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - private final static int stripesPerBlock = 4; - static int blockSize = cellSize * stripesPerBlock; - static int numDNs = dataBlocks + parityBlocks + 2; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock; +public class TestWriteReadStripedFile { private static MiniDFSCluster cluster; - private static Configuration conf; private static FileSystem fs; - - private static Random r= new Random(); + private static Configuration conf; @BeforeClass public static void setup() throws IOException { - conf = new Configuration(); + conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null, cellSize); + cluster.getFileSystem().getClient().createErasureCodingZone("/", + null, cellSize); fs = cluster.getFileSystem(); } @AfterClass - public static void tearDown() { + public static void tearDown() throws IOException { if (cluster != null) { cluster.shutdown(); } @@ -152,47 +144,21 @@ public void testFileMoreThanABlockGroup3() throws IOException { + cellSize + 123); } - private byte[] generateBytes(int cnt) { - byte[] bytes = new byte[cnt]; - for (int i = 0; i < cnt; i++) { - bytes[i] = getByte(i); - } - return bytes; - } - - private int readAll(FSDataInputStream in, byte[] buf) throws IOException { - int readLen = 0; - int ret; - do { - ret = in.read(buf, readLen, buf.length - readLen); - if (ret > 0) { - readLen += ret; - } - } while (ret >= 0 && readLen < buf.length); - return readLen; - } - - private byte getByte(long pos) { - final int mod = 29; - return (byte) (pos % mod + 1); - } - private void assertSeekAndRead(FSDataInputStream fsdis, int pos, int writeBytes) throws IOException { fsdis.seek(pos); byte[] buf = new byte[writeBytes]; - int readLen = readAll(fsdis, buf); + int readLen = StripedFileTestUtil.readAll(fsdis, buf); Assert.assertEquals(readLen, writeBytes - pos); for (int i = 0; i < readLen; i++) { Assert.assertEquals("Byte at " + i + " should be the same", - getByte(pos + i), buf[i]); + StripedFileTestUtil.getByte(pos + i), buf[i]); } } private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) throws IOException { - - final byte[] expected = generateBytes(fileLength); + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); Path srcPath = new Path(src); DFSTestUtil.writeFile(fs, srcPath, new String(expected)); @@ -215,7 +181,7 @@ private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) public void testWriteReadUsingWebHdfs() throws Exception { int fileLength = blockSize * dataBlocks + cellSize + 123; - final byte[] expected = generateBytes(fileLength); + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME); Path srcPath = new Path("/testWriteReadUsingWebHdfs_stripe"); @@ -231,7 +197,6 @@ public void testWriteReadUsingWebHdfs() throws Exception { verifySeek(fs, srcPath, fileLength); verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); //webhdfs doesn't support bytebuffer read - } void verifyLength(FileSystem fs, Path srcPath, int fileLength) @@ -243,152 +208,105 @@ void verifyLength(FileSystem fs, Path srcPath, int fileLength) void verifyPread(FileSystem fs, Path srcPath, int fileLength, byte[] expected, byte[] buf) throws IOException { - FSDataInputStream in = fs.open(srcPath); - int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102, - cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102, - cellSize * dataBlocks, fileLength - 102, fileLength - 1}; - for (int startOffset : startOffsets) { - startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); - int remaining = fileLength - startOffset; - in.readFully(startOffset, buf, 0, remaining); - for (int i = 0; i < remaining; i++) { - Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + - "same", - expected[startOffset + i], buf[i]); + try (FSDataInputStream in = fs.open(srcPath)) { + int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102, + cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102, + cellSize * dataBlocks, fileLength - 102, fileLength - 1}; + for (int startOffset : startOffsets) { + startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); + int remaining = fileLength - startOffset; + in.readFully(startOffset, buf, 0, remaining); + for (int i = 0; i < remaining; i++) { + Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + + "same", expected[startOffset + i], buf[i]); + } } } - in.close(); } void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, byte[] expected, byte[] buf) throws IOException { - FSDataInputStream in = fs.open(srcPath); - final byte[] result = new byte[fileLength]; - int readLen = 0; - int ret; - do { - ret = in.read(buf, 0, buf.length); - if (ret > 0) { + try (FSDataInputStream in = fs.open(srcPath)) { + final byte[] result = new byte[fileLength]; + int readLen = 0; + int ret; + while ((ret = in.read(buf, 0, buf.length)) >= 0) { System.arraycopy(buf, 0, result, readLen, ret); readLen += ret; } - } while (ret >= 0); - Assert.assertEquals("The length of file should be the same to write size", - fileLength, readLen); - Assert.assertArrayEquals(expected, result); - in.close(); + Assert.assertEquals("The length of file should be the same to write size", + fileLength, readLen); + Assert.assertArrayEquals(expected, result); + } } void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, byte[] expected, ByteBuffer buf) throws IOException { - FSDataInputStream in = fs.open(srcPath); - ByteBuffer result = ByteBuffer.allocate(fileLength); - int readLen = 0; - int ret; - do { - ret = in.read(buf); - if (ret > 0) { + try (FSDataInputStream in = fs.open(srcPath)) { + ByteBuffer result = ByteBuffer.allocate(fileLength); + int readLen = 0; + int ret; + while ((ret = in.read(buf)) >= 0) { readLen += ret; buf.flip(); result.put(buf); buf.clear(); } - } while (ret >= 0); - readLen = readLen >= 0 ? readLen : 0; - Assert.assertEquals("The length of file should be the same to write size", - fileLength, readLen); - Assert.assertArrayEquals(expected, result.array()); - in.close(); + Assert.assertEquals("The length of file should be the same to write size", + fileLength, readLen); + Assert.assertArrayEquals(expected, result.array()); + } } void verifySeek(FileSystem fs, Path srcPath, int fileLength) throws IOException { - FSDataInputStream in = fs.open(srcPath); - // seek to 1/2 of content - int pos = fileLength / 2; - assertSeekAndRead(in, pos, fileLength); - - // seek to 1/3 of content - pos = fileLength / 3; - assertSeekAndRead(in, pos, fileLength); - - // seek to 0 pos - pos = 0; - assertSeekAndRead(in, pos, fileLength); - - if (fileLength > cellSize) { - // seek to cellSize boundary - pos = cellSize - 1; + try (FSDataInputStream in = fs.open(srcPath)) { + // seek to 1/2 of content + int pos = fileLength / 2; assertSeekAndRead(in, pos, fileLength); - } - if (fileLength > cellSize * dataBlocks) { - // seek to striped cell group boundary - pos = cellSize * dataBlocks - 1; + // seek to 1/3 of content + pos = fileLength / 3; assertSeekAndRead(in, pos, fileLength); - } - if (fileLength > blockSize * dataBlocks) { - // seek to striped block group boundary - pos = blockSize * dataBlocks - 1; + // seek to 0 pos + pos = 0; assertSeekAndRead(in, pos, fileLength); - } - if(!(in.getWrappedStream() instanceof ByteRangeInputStream)){ - try { - in.seek(-1); - Assert.fail("Should be failed if seek to negative offset"); - } catch (EOFException e) { - // expected + if (fileLength > cellSize) { + // seek to cellSize boundary + pos = cellSize - 1; + assertSeekAndRead(in, pos, fileLength); } - try { - in.seek(fileLength + 1); - Assert.fail("Should be failed if seek after EOF"); - } catch (EOFException e) { - // expected + if (fileLength > cellSize * dataBlocks) { + // seek to striped cell group boundary + pos = cellSize * dataBlocks - 1; + assertSeekAndRead(in, pos, fileLength); } - } - in.close(); - } - @Test - public void testWritePreadWithDNFailure() throws IOException { - final int failedDNIdx = 2; - final int length = cellSize * (dataBlocks + 2); - Path testPath = new Path("/foo"); - final byte[] bytes = generateBytes(length); - DFSTestUtil.writeFile(fs, testPath, new String(bytes)); - - // shut down the DN that holds the last internal data block - BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, - cellSize); - String name = (locs[0].getNames())[failedDNIdx]; - for (DataNode dn : cluster.getDataNodes()) { - int port = dn.getXferPort(); - if (name.contains(Integer.toString(port))) { - dn.shutdown(); - break; + if (fileLength > blockSize * dataBlocks) { + // seek to striped block group boundary + pos = blockSize * dataBlocks - 1; + assertSeekAndRead(in, pos, fileLength); } - } - // pread - int startOffsetInFile = cellSize * 5; - try (FSDataInputStream fsdis = fs.open(testPath)) { - byte[] buf = new byte[length]; - int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length); - Assert.assertEquals("The length of file should be the same to write size", - length - startOffsetInFile, readLen); - - byte[] expected = new byte[readLen]; - for (int i = startOffsetInFile; i < length; i++) { - expected[i - startOffsetInFile] = getByte(i); - } - for (int i = startOffsetInFile; i < length; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", - expected[i - startOffsetInFile], buf[i - startOffsetInFile]); + if (!(in.getWrappedStream() instanceof ByteRangeInputStream)) { + try { + in.seek(-1); + Assert.fail("Should be failed if seek to negative offset"); + } catch (EOFException e) { + // expected + } + + try { + in.seek(fileLength + 1); + Assert.fail("Should be failed if seek after EOF"); + } catch (EOFException e) { + // expected + } } } } From 5f15084bd530865d3e2641b709665b5b7971a74d Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 2 Jun 2015 16:14:08 -0700 Subject: [PATCH 147/212] HDFS-8453. Erasure coding: properly handle start offset for internal blocks in a block group. Contributed by Zhe Zhang. --- .../hadoop/hdfs/protocol/HdfsConstants.java | 2 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../apache/hadoop/hdfs/DFSInputStream.java | 48 +++++++++++-------- .../hadoop/hdfs/DFSStripedInputStream.java | 45 ++++++++--------- .../hadoop/hdfs/util/StripedBlockUtil.java | 5 +- .../hdfs/TestDFSStripedInputStream.java | 4 +- .../hdfs/TestReadStripedFileWithDecoding.java | 24 ++++++---- 7 files changed, 72 insertions(+), 59 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java index 32ca81c326749..a527e233de59f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java @@ -84,7 +84,7 @@ public final class HdfsConstants { public static final byte NUM_DATA_BLOCKS = 6; public static final byte NUM_PARITY_BLOCKS = 3; // The chunk size for striped block which is used by erasure coding - public static final int BLOCK_STRIPED_CELL_SIZE = 256 * 1024; + public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024; // SafeMode actions public enum SafeModeAction { diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index fa0a8e2c543ba..278f897e42961 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -274,3 +274,6 @@ HDFS-8517. Fix a decoding issue in stripped block recovering in client side. (Kai Zheng via jing9) + + HDFS-8453. Erasure coding: properly handle start offset for internal blocks + in a block group. (Zhe Zhang via jing9) 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 0d51a5752da13..6102edfe92a44 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 @@ -1016,7 +1016,7 @@ private DNAddrPair chooseDataNode(LocatedBlock block, } deadNodes.clear(); //2nd option is to remove only nodes[blockId] openInfo(); - block = getBlockAt(block.getStartOffset()); + block = refreshLocatedBlock(block); failures++; } } @@ -1088,15 +1088,15 @@ private static String getBestNodeDNAddrPairErrorString( return errMsgr.toString(); } - protected void fetchBlockByteRange(long blockStartOffset, long start, long end, + protected void fetchBlockByteRange(LocatedBlock block, long start, long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { - LocatedBlock block = getBlockAt(blockStartOffset); + block = refreshLocatedBlock(block); while (true) { DNAddrPair addressPair = chooseDataNode(block, null); try { - actualGetFromOneDataNode(addressPair, blockStartOffset, start, end, + actualGetFromOneDataNode(addressPair, block, start, end, buf, offset, corruptedBlockMap); return; } catch (IOException e) { @@ -1107,7 +1107,7 @@ protected void fetchBlockByteRange(long blockStartOffset, long start, long end, } private Callable getFromOneDataNode(final DNAddrPair datanode, - final long blockStartOffset, final long start, final long end, + final LocatedBlock block, final long start, final long end, final ByteBuffer bb, final Map> corruptedBlockMap, final int hedgedReadId) { @@ -1120,7 +1120,7 @@ public ByteBuffer call() throws Exception { TraceScope scope = Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan); try { - actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf, + actualGetFromOneDataNode(datanode, block, start, end, buf, offset, corruptedBlockMap); return bb; } finally { @@ -1134,18 +1134,18 @@ public ByteBuffer call() throws Exception { * Used when reading contiguous blocks */ private void actualGetFromOneDataNode(final DNAddrPair datanode, - long blockStartOffset, final long start, final long end, byte[] buf, + LocatedBlock block, final long start, final long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { final int length = (int) (end - start + 1); - actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf, + actualGetFromOneDataNode(datanode, block, start, end, buf, new int[]{offset}, new int[]{length}, corruptedBlockMap); } /** * Read data from one DataNode. * @param datanode the datanode from which to read data - * @param blockStartOffset starting offset in the file + * @param block the located block containing the requested data * @param startInBlk the startInBlk offset of the block * @param endInBlk the endInBlk offset of the block * @param buf the given byte array into which the data is read @@ -1157,7 +1157,7 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, * block replica */ void actualGetFromOneDataNode(final DNAddrPair datanode, - long blockStartOffset, final long startInBlk, final long endInBlk, + LocatedBlock block, final long startInBlk, final long endInBlk, byte[] buf, int[] offsets, int[] lengths, Map> corruptedBlockMap) throws IOException { @@ -1171,7 +1171,7 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, // cached block locations may have been updated by chooseDataNode() // or fetchBlockAt(). Always get the latest list of locations at the // start of the loop. - LocatedBlock block = getBlockAt(blockStartOffset); + block = refreshLocatedBlock(block); BlockReader reader = null; try { DFSClientFaultInjector.get().fetchFromDatanodeException(); @@ -1227,6 +1227,17 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, } } + /** + * Refresh cached block locations. + * @param block The currently cached block locations + * @return Refreshed block locations + * @throws IOException + */ + protected LocatedBlock refreshLocatedBlock(LocatedBlock block) + throws IOException { + return getBlockAt(block.getStartOffset()); + } + /** * This method verifies that the read portions are valid and do not overlap * with each other. @@ -1250,7 +1261,7 @@ private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) { * 'hedged' read if the first read is taking longer than configured amount of * time. We then wait on which ever read returns first. */ - private void hedgedFetchBlockByteRange(long blockStartOffset, long start, + private void hedgedFetchBlockByteRange(LocatedBlock block, long start, long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { @@ -1263,7 +1274,7 @@ private void hedgedFetchBlockByteRange(long blockStartOffset, long start, ByteBuffer bb = null; int len = (int) (end - start + 1); int hedgedReadId = 0; - LocatedBlock block = getBlockAt(blockStartOffset); + block = refreshLocatedBlock(block); while (true) { // see HDFS-6591, this metric is used to verify/catch unnecessary loops hedgedReadOpsLoopNumForTesting++; @@ -1275,7 +1286,7 @@ private void hedgedFetchBlockByteRange(long blockStartOffset, long start, chosenNode = chooseDataNode(block, ignored); bb = ByteBuffer.wrap(buf, offset, len); Callable getFromDataNodeCallable = getFromOneDataNode( - chosenNode, block.getStartOffset(), start, end, bb, + chosenNode, block, start, end, bb, corruptedBlockMap, hedgedReadId++); Future firstRequest = hedgedService .submit(getFromDataNodeCallable); @@ -1312,7 +1323,7 @@ private void hedgedFetchBlockByteRange(long blockStartOffset, long start, } bb = ByteBuffer.allocate(len); Callable getFromDataNodeCallable = getFromOneDataNode( - chosenNode, block.getStartOffset(), start, end, bb, + chosenNode, block, start, end, bb, corruptedBlockMap, hedgedReadId++); Future oneMoreRequest = hedgedService .submit(getFromDataNodeCallable); @@ -1466,12 +1477,11 @@ private int pread(long position, byte[] buffer, int offset, int length) long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart); try { if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) { - hedgedFetchBlockByteRange(blk.getStartOffset(), targetStart, + hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap); } else { - fetchBlockByteRange(blk.getStartOffset(), targetStart, - targetStart + bytesToRead - 1, buffer, offset, - corruptedBlockMap); + fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1, + buffer, offset, corruptedBlockMap); } } finally { // Check and report if any block replicas are corrupted. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 228368ba4925f..2e26cca094afa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.ByteBufferPool; @@ -470,22 +471,17 @@ private int copy(ReaderStrategy strategy, int offset, int length) { } /** - * | <--------- LocatedStripedBlock (ID = 0) ---------> | - * LocatedBlock (0) | LocatedBlock (1) | LocatedBlock (2) - * ^ - * offset - * On a striped file, the super method {@link DFSInputStream#getBlockAt} - * treats a striped block group as a single {@link LocatedBlock} object, - * which includes target in its range. This method adds the logic of: - * 1. Analyzing the index of required block based on offset - * 2. Parsing the block group to obtain the block location on that index + * The super method {@link DFSInputStream#refreshLocatedBlock} refreshes + * cached LocatedBlock by executing {@link DFSInputStream#getBlockAt} again. + * This method extends the logic by first remembering the index of the + * internal block, and re-parsing the refreshed block group with the same + * index. */ @Override - protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { - LocatedBlock lb = getBlockGroupAt(blkStartOffset); - - int idx = (int) ((blkStartOffset - lb.getStartOffset()) - % (dataBlkNum + parityBlkNum)); + protected LocatedBlock refreshLocatedBlock(LocatedBlock block) + throws IOException { + int idx = BlockIdManager.getBlockIndex(block.getBlock().getLocalBlock()); + LocatedBlock lb = getBlockGroupAt(block.getStartOffset()); // If indexing information is returned, iterate through the index array // to find the entry for position idx in the group LocatedStripedBlock lsb = (LocatedStripedBlock) lb; @@ -496,10 +492,11 @@ protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException { } } if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("getBlockAt for striped blocks, offset=" - + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx); + DFSClient.LOG.debug("refreshLocatedBlock for striped blocks, offset=" + + block.getStartOffset() + ". Obtained block " + lb + ", idx=" + idx); } - return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize, dataBlkNum, idx); + return StripedBlockUtil.constructInternalBlock( + lsb, i, cellSize, dataBlkNum, idx); } private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException { @@ -513,12 +510,12 @@ private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException { * Real implementation of pread. */ @Override - protected void fetchBlockByteRange(long blockStartOffset, long start, + protected void fetchBlockByteRange(LocatedBlock block, long start, long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { // Refresh the striped block group - LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset); + LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset()); AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, cellSize, blockGroup, start, end, buf, offset); @@ -622,9 +619,9 @@ private void fetchOneStripingChunk(Map, Integer> futures, StripingChunk chunk = alignedStripe.chunks[index]; chunk.state = StripingChunk.PENDING; Callable readCallable = getFromOneDataNode(dnAddr, - block.getStartOffset(), alignedStripe.getOffsetInBlock(), - alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, chunk.buf, - chunk.getOffsets(), chunk.getLengths(), + block, alignedStripe.getOffsetInBlock(), + alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, + chunk.buf, chunk.getOffsets(), chunk.getLengths(), corruptedBlockMap, index); Future getFromDNRequest = service.submit(readCallable); if (DFSClient.LOG.isDebugEnabled()) { @@ -637,7 +634,7 @@ private void fetchOneStripingChunk(Map, Integer> futures, } private Callable getFromOneDataNode(final DNAddrPair datanode, - final long blockStartOffset, final long start, final long end, + final LocatedBlock block, final long start, final long end, final byte[] buf, final int[] offsets, final int[] lengths, final Map> corruptedBlockMap, final int hedgedReadId) { @@ -648,7 +645,7 @@ public Void call() throws Exception { TraceScope scope = Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan); try { - actualGetFromOneDataNode(datanode, blockStartOffset, start, + actualGetFromOneDataNode(datanode, block, start, end, buf, offsets, lengths, corruptedBlockMap); } finally { scope.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 80321ef9590ab..1db2045741f03 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -105,16 +105,15 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, final ExtendedBlock blk = constructInternalBlock( bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); - final long offset = bg.getStartOffset() + idxInBlockGroup * (long) cellSize; if (idxInReturnedLocs < bg.getLocations().length) { return new LocatedBlock(blk, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, - offset, bg.isCorrupt(), null); + bg.getStartOffset(), bg.isCorrupt(), null); } else { return new LocatedBlock(blk, null, null, null, - offset, bg.isCorrupt(), null); + bg.getStartOffset(), bg.isCorrupt(), null); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index b64e690e1eb33..de43441d671a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -87,7 +87,7 @@ public void tearDown() { * Test {@link DFSStripedInputStream#getBlockAt(long)} */ @Test - public void testGetBlock() throws Exception { + public void testRefreshBlock() throws Exception { final int numBlocks = 4; DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, NUM_STRIPE_PER_BLOCK, false); @@ -102,7 +102,7 @@ public void testGetBlock() throws Exception { LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); for (int j = 0; j < DATA_BLK_NUM; j++) { - LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset()); + LocatedBlock refreshed = in.refreshLocatedBlock(blks[j]); assertEquals(blks[j].getBlock(), refreshed.getBlock()); assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset()); assertArrayEquals(blks[j].getLocations(), refreshed.getLocations()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 7397caf470218..a28f88ef8bbd4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -59,20 +59,24 @@ public void tearDown() throws IOException { @Test public void testWritePreadWithDNFailure1() throws IOException { - testWritePreadWithDNFailure("/foo", 0); + testWritePreadWithDNFailure("/foo", cellSize * (dataBlocks + 2), 0); } @Test public void testWritePreadWithDNFailure2() throws IOException { - testWritePreadWithDNFailure("/foo", cellSize * 5); + testWritePreadWithDNFailure("/foo", cellSize * (dataBlocks + 2), cellSize * 5); } - private void testWritePreadWithDNFailure(String file, int startOffsetInFile) + @Test + public void testWritePreadWithDNFailure3() throws IOException { + testWritePreadWithDNFailure("/foo", cellSize * dataBlocks, 0); + } + + private void testWritePreadWithDNFailure(String file, int fileSize, int startOffsetInFile) throws IOException { final int failedDNIdx = 2; - final int length = cellSize * (dataBlocks + 2); Path testPath = new Path(file); - final byte[] bytes = StripedFileTestUtil.generateBytes(length); + final byte[] bytes = StripedFileTestUtil.generateBytes(fileSize); DFSTestUtil.writeFile(fs, testPath, bytes); // shut down the DN that holds the last internal data block @@ -89,17 +93,17 @@ private void testWritePreadWithDNFailure(String file, int startOffsetInFile) // pread try (FSDataInputStream fsdis = fs.open(testPath)) { - byte[] buf = new byte[length]; + byte[] buf = new byte[fileSize]; int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length); - Assert.assertEquals("The length of file should be the same to write size", - length - startOffsetInFile, readLen); + Assert.assertEquals("The fileSize of file should be the same to write size", + fileSize - startOffsetInFile, readLen); byte[] expected = new byte[readLen]; - for (int i = startOffsetInFile; i < length; i++) { + for (int i = startOffsetInFile; i < fileSize; i++) { expected[i - startOffsetInFile] = StripedFileTestUtil.getByte(i); } - for (int i = startOffsetInFile; i < length; i++) { + for (int i = startOffsetInFile; i < fileSize; i++) { Assert.assertEquals("Byte at " + i + " should be the same", expected[i - startOffsetInFile], buf[i - startOffsetInFile]); } From 673280df24f0228bf01777035ceeab8807da8c40 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 3 Jun 2015 11:51:58 -0700 Subject: [PATCH 148/212] HDFS-7621. Erasure Coding: update the Balancer/Mover data migration logic. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/protocolPB/PBHelper.java | 21 ++- .../hdfs/server/balancer/Dispatcher.java | 148 +++++++++++++----- .../server/blockmanagement/BlockManager.java | 26 ++- .../hadoop/hdfs/server/mover/Mover.java | 38 ++++- .../server/protocol/BlocksWithLocations.java | 25 +++ .../hadoop-hdfs/src/main/proto/hdfs.proto | 3 + .../org/apache/hadoop/hdfs/DFSTestUtil.java | 15 ++ .../hadoop/hdfs/protocolPB/TestPBHelper.java | 51 ++++-- .../hdfs/server/balancer/TestBalancer.java | 76 +++++++++ .../hadoop/hdfs/server/mover/TestMover.java | 124 ++++++++++++++- 11 files changed, 452 insertions(+), 78 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 278f897e42961..511ebecc20ae9 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -277,3 +277,6 @@ HDFS-8453. Erasure coding: properly handle start offset for internal blocks in a block group. (Zhe Zhang via jing9) + + HDFS-7621. Erasure Coding: update the Balancer/Mover data migration logic. + (Walter Su via zhz) 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 b2415fa83bf2b..0bfc3bbf3abcb 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 @@ -211,6 +211,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; @@ -458,22 +459,32 @@ public static StripedBlockProto convert(BlockInfoStriped blk) { } public static BlockWithLocationsProto convert(BlockWithLocations blk) { - return BlockWithLocationsProto.newBuilder() - .setBlock(convert(blk.getBlock())) + BlockWithLocationsProto.Builder builder = BlockWithLocationsProto + .newBuilder().setBlock(convert(blk.getBlock())) .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids())) .addAllStorageUuids(Arrays.asList(blk.getStorageIDs())) - .addAllStorageTypes(convertStorageTypes(blk.getStorageTypes())) - .build(); + .addAllStorageTypes(convertStorageTypes(blk.getStorageTypes())); + if (blk instanceof StripedBlockWithLocations) { + StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk; + builder.setIndices(getByteString(sblk.getIndices())); + builder.setDataBlockNum(sblk.getDataBlockNum()); + } + return builder.build(); } public static BlockWithLocations convert(BlockWithLocationsProto b) { final List datanodeUuids = b.getDatanodeUuidsList(); final List storageUuids = b.getStorageUuidsList(); final List storageTypes = b.getStorageTypesList(); - return new BlockWithLocations(convert(b.getBlock()), + BlockWithLocations blk = new BlockWithLocations(convert(b.getBlock()), datanodeUuids.toArray(new String[datanodeUuids.size()]), storageUuids.toArray(new String[storageUuids.size()]), convertStorageTypes(storageTypes, storageUuids.size())); + if (b.hasIndices()) { + blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(), + (short) b.getDataBlockNum()); + } + return blk; } public static BlocksWithLocationsProto convert(BlocksWithLocations blks) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java index 4a8f40fa33b64..930001aa51d86 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.balancer; import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; @@ -52,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; @@ -65,6 +67,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; @@ -124,18 +127,17 @@ private static class GlobalBlockMap { private final Map map = new HashMap(); /** - * Get the block from the map; - * if the block is not found, create a new block and put it in the map. + * Put block in the map if it's not found + * @return the block which be put in the map the first time */ - private DBlock get(Block b) { - DBlock block = map.get(b); - if (block == null) { - block = new DBlock(b); - map.put(b, block); + private DBlock putIfAbsent(Block blk, DBlock dblk) { + if (!map.containsKey(blk)) { + map.put(blk, dblk); + return dblk; } - return block; + return map.get(blk); } - + /** Remove all blocks except for the moved blocks. */ private void removeAllButRetain(MovedBlocks movedBlocks) { for (Iterator i = map.keySet().iterator(); i.hasNext();) { @@ -176,9 +178,9 @@ public Collection values() { } } - /** This class keeps track of a scheduled block move */ + /** This class keeps track of a scheduled reportedBlock move */ public class PendingMove { - private DBlock block; + private DBlock reportedBlock; private Source source; private DDatanode proxySource; private StorageGroup target; @@ -190,7 +192,7 @@ private PendingMove(Source source, StorageGroup target) { @Override public String toString() { - final Block b = block != null ? block.getBlock() : null; + final Block b = reportedBlock != null ? reportedBlock.getBlock() : null; String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ") : " "; return bStr + "from " + source.getDisplayName() + " to " + target @@ -199,8 +201,8 @@ public String toString() { } /** - * Choose a block & a proxy source for this pendingMove whose source & - * target have already been chosen. + * Choose a good block/blockGroup from source & Get reportedBlock from + * the block & Choose a proxy source for the reportedBlock. * * @return true if a block and its proxy are chosen; false otherwise */ @@ -224,7 +226,11 @@ private boolean markMovedIfGoodBlock(DBlock block, StorageType targetStorageType synchronized (block) { synchronized (movedBlocks) { if (isGoodBlockCandidate(source, target, targetStorageType, block)) { - this.block = block; + if (block instanceof DBlockStriped) { + reportedBlock = ((DBlockStriped) block).getInternalBlock(source); + } else { + reportedBlock = block; + } if (chooseProxySource()) { movedBlocks.put(block); if (LOG.isDebugEnabled()) { @@ -251,7 +257,7 @@ private boolean chooseProxySource() { } // if node group is supported, first try add nodes in the same node group if (cluster.isNodeGroupAware()) { - for (StorageGroup loc : block.getLocations()) { + for (StorageGroup loc : reportedBlock.getLocations()) { if (cluster.isOnSameNodeGroup(loc.getDatanodeInfo(), targetDN) && addTo(loc)) { return true; @@ -259,13 +265,13 @@ && addTo(loc)) { } } // check if there is replica which is on the same rack with the target - for (StorageGroup loc : block.getLocations()) { + for (StorageGroup loc : reportedBlock.getLocations()) { if (cluster.isOnSameRack(loc.getDatanodeInfo(), targetDN) && addTo(loc)) { return true; } } // find out a non-busy replica - for (StorageGroup loc : block.getLocations()) { + for (StorageGroup loc : reportedBlock.getLocations()) { if (addTo(loc)) { return true; } @@ -273,7 +279,7 @@ && addTo(loc)) { return false; } - /** add to a proxy source for specific block movement */ + /** add to a proxy source for specific reportedBlock movement */ private boolean addTo(StorageGroup g) { final DDatanode dn = g.getDDatanode(); if (dn.addPendingBlock(this)) { @@ -288,6 +294,7 @@ private void dispatch() { if (LOG.isDebugEnabled()) { LOG.debug("Start moving " + this); } + assert !(reportedBlock instanceof DBlockStriped); Socket sock = new Socket(); DataOutputStream out = null; @@ -302,7 +309,7 @@ private void dispatch() { OutputStream unbufOut = sock.getOutputStream(); InputStream unbufIn = sock.getInputStream(); ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(), - block.getBlock()); + reportedBlock.getBlock()); final KeyManager km = nnc.getKeyManager(); Token accessToken = km.getAccessToken(eb); IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut, @@ -316,7 +323,7 @@ private void dispatch() { sendRequest(out, eb, accessToken); receiveResponse(in); - nnc.getBytesMoved().addAndGet(block.getNumBytes()); + nnc.getBytesMoved().addAndGet(reportedBlock.getNumBytes()); LOG.info("Successfully moved " + this); } catch (IOException e) { LOG.warn("Failed to move " + this + ": " + e.getMessage()); @@ -344,14 +351,14 @@ private void dispatch() { } } - /** Send a block replace request to the output stream */ + /** Send a reportedBlock replace request to the output stream */ private void sendRequest(DataOutputStream out, ExtendedBlock eb, Token accessToken) throws IOException { new Sender(out).replaceBlock(eb, target.storageType, accessToken, source.getDatanodeInfo().getDatanodeUuid(), proxySource.datanode); } - /** Receive a block copy response from the input stream */ + /** Receive a reportedBlock copy response from the input stream */ private void receiveResponse(DataInputStream in) throws IOException { BlockOpResponseProto response = BlockOpResponseProto.parseFrom(vintPrefixed(in)); @@ -359,13 +366,13 @@ private void receiveResponse(DataInputStream in) throws IOException { // read intermediate responses response = BlockOpResponseProto.parseFrom(vintPrefixed(in)); } - String logInfo = "block move is failed"; + String logInfo = "reportedBlock move is failed"; DataTransferProtoUtil.checkBlockOpStatus(response, logInfo); } /** reset the object */ private void reset() { - block = null; + reportedBlock = null; source = null; proxySource = null; target = null; @@ -377,6 +384,44 @@ public static class DBlock extends MovedBlocks.Locations { public DBlock(Block block) { super(block); } + + public long getNumBytes(StorageGroup storage) { + return super.getNumBytes(); + } + } + + public static class DBlockStriped extends DBlock { + + final byte[] indices; + final short dataBlockNum; + + public DBlockStriped(Block block, byte[] indices, short dataBlockNum) { + super(block); + this.indices = indices; + this.dataBlockNum = dataBlockNum; + } + + public DBlock getInternalBlock(StorageGroup storage) { + int idxInLocs = locations.indexOf(storage); + if (idxInLocs == -1) { + return null; + } + byte idxInGroup = indices[idxInLocs]; + long blkId = getBlock().getBlockId() + idxInGroup; + long numBytes = getInternalBlockLength(getNumBytes(), + HdfsConstants.BLOCK_STRIPED_CELL_SIZE, dataBlockNum, idxInGroup); + Block blk = new Block(getBlock()); + blk.setBlockId(blkId); + blk.setNumBytes(numBytes); + DBlock dblk = new DBlock(blk); + dblk.addLocation(storage); + return dblk; + } + + @Override + public long getNumBytes(StorageGroup storage) { + return getInternalBlock(storage).getNumBytes(); + } } /** The class represents a desired move. */ @@ -452,7 +497,7 @@ synchronized void resetScheduledSize() { private PendingMove addPendingMove(DBlock block, final PendingMove pm) { if (getDDatanode().addPendingBlock(pm)) { if (pm.markMovedIfGoodBlock(block, getStorageType())) { - incScheduledSize(pm.block.getNumBytes()); + incScheduledSize(pm.reportedBlock.getNumBytes()); return pm; } else { getDDatanode().removePendingBlock(pm); @@ -612,19 +657,34 @@ Iterator getBlockIterator() { */ private long getBlockList() throws IOException { final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive); - final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size); + final BlocksWithLocations newBlksLocs = + nnc.getBlocks(getDatanodeInfo(), size); long bytesReceived = 0; - for (BlockWithLocations blk : newBlocks.getBlocks()) { - bytesReceived += blk.getBlock().getNumBytes(); + for (BlockWithLocations blkLocs : newBlksLocs.getBlocks()) { + + DBlock block; + if (blkLocs instanceof StripedBlockWithLocations) { + StripedBlockWithLocations sblkLocs = + (StripedBlockWithLocations) blkLocs; + // approximate size + bytesReceived += sblkLocs.getBlock().getNumBytes() / + sblkLocs.getDataBlockNum(); + block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(), + sblkLocs.getDataBlockNum()); + } else{ + bytesReceived += blkLocs.getBlock().getNumBytes(); + block = new DBlock(blkLocs.getBlock()); + } + synchronized (globalBlocks) { - final DBlock block = globalBlocks.get(blk.getBlock()); + block = globalBlocks.putIfAbsent(blkLocs.getBlock(), block); synchronized (block) { block.clearLocations(); // update locations - final String[] datanodeUuids = blk.getDatanodeUuids(); - final StorageType[] storageTypes = blk.getStorageTypes(); + final String[] datanodeUuids = blkLocs.getDatanodeUuids(); + final StorageType[] storageTypes = blkLocs.getStorageTypes(); for (int i = 0; i < datanodeUuids.length; i++) { final StorageGroup g = storageGroupMap.get( datanodeUuids[i], storageTypes[i]); @@ -661,6 +721,8 @@ private boolean isGoodBlockCandidate(DBlock block) { * target throttling has been considered. They are chosen only when they * have the capacity to support this block move. The block should be * dispatched immediately after this method is returned. + * If the block is a block group. Only the internal block on this source + * will be dispatched. * * @return a move that's good for the source to dispatch immediately. */ @@ -672,7 +734,7 @@ private PendingMove chooseNextMove() { if (target.addPendingBlock(pendingBlock)) { // target is not busy, so do a tentative block allocation if (pendingBlock.chooseBlockAndProxy()) { - long blockSize = pendingBlock.block.getNumBytes(); + long blockSize = pendingBlock.reportedBlock.getNumBytes(this); incScheduledSize(-blockSize); task.size -= blockSize; if (task.size == 0) { @@ -744,7 +806,7 @@ private void dispatchBlocks() { blocksToReceive -= getBlockList(); continue; } catch (IOException e) { - LOG.warn("Exception while getting block list", e); + LOG.warn("Exception while getting reportedBlock list", e); return; } } else { @@ -883,7 +945,7 @@ public DDatanode newDatanode(DatanodeInfo datanode) { } public void executePendingMove(final PendingMove p) { - // move the block + // move the reportedBlock moveExecutor.execute(new Runnable() { @Override public void run() { @@ -928,17 +990,17 @@ public void run() { } } - // wait for all block moving to be done + // wait for all reportedBlock moving to be done waitForMoveCompletion(targets); return getBytesMoved() - bytesLastMoved; } - /** The sleeping period before checking if block move is completed again */ + /** The sleeping period before checking if reportedBlock move is completed again */ static private long blockMoveWaitTime = 30000L; /** - * Wait for all block move confirmations. + * Wait for all reportedBlock move confirmations. * @return true if there is failed move execution */ public static boolean waitForMoveCompletion( @@ -965,10 +1027,10 @@ public static boolean waitForMoveCompletion( } /** - * Decide if the block is a good candidate to be moved from source to target. - * A block is a good candidate if + * Decide if the block/blockGroup is a good candidate to be moved from source + * to target. A block is a good candidate if * 1. the block is not in the process of being moved/has not been moved; - * 2. the block does not have a replica on the target; + * 2. the block does not have a replica/internalBlock on the target; * 3. doing the move does not reduce the number of racks that the block has */ private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target, @@ -985,7 +1047,7 @@ private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target, } final DatanodeInfo targetDatanode = target.getDatanodeInfo(); if (source.getDatanodeInfo().equals(targetDatanode)) { - // the block is moved inside same DN + // the reportedBlock is moved inside same DN return true; } @@ -1068,7 +1130,7 @@ void reset(Configuration conf) { movedBlocks.cleanup(); } - /** set the sleeping period for block move completion check */ + /** set the sleeping period for reportedBlock move completion check */ @VisibleForTesting public static void setBlockMoveWaitTime(long time) { blockMoveWaitTime = time; 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 32757f925bd76..48a1b35d45d6e 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 @@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; @@ -3265,9 +3266,10 @@ public void removeStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) { /** * Get all valid locations of the block & add the block to results - * return the length of the added block; 0 if the block is not added + * @return the length of the added block; 0 if the block is not added. If the + * added block is a block group, return its approximate internal block size */ - private long addBlock(Block block, List results) { + private long addBlock(BlockInfo block, List results) { final List locations = getValidLocations(block); if(locations.size() == 0) { return 0; @@ -3281,9 +3283,23 @@ private long addBlock(Block block, List results) { storageIDs[i] = s.getStorageID(); storageTypes[i] = s.getStorageType(); } - results.add(new BlockWithLocations(block, datanodeUuids, storageIDs, - storageTypes)); - return block.getNumBytes(); + BlockWithLocations blkWithLocs = new BlockWithLocations(block, + datanodeUuids, storageIDs, storageTypes); + if(block.isStriped()) { + BlockInfoStriped blockStriped = (BlockInfoStriped) block; + byte[] indices = new byte[locations.size()]; + for (int i = 0; i < locations.size(); i++) { + indices[i] = + (byte) blockStriped.getStorageBlockIndex(locations.get(i)); + } + results.add(new StripedBlockWithLocations(blkWithLocs, indices, + blockStriped.getDataBlockNum())); + // approximate size + return block.getNumBytes() / blockStriped.getDataBlockNum(); + }else{ + results.add(blkWithLocs); + return block.getNumBytes(); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java index 8715ce4cd9414..ddfd1ea546c57 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java @@ -46,12 +46,15 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.NetworkTopology; 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 static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; + import java.io.BufferedReader; import java.io.FileInputStream; import java.io.IOException; @@ -176,8 +179,20 @@ private ExitStatus run() { } } - DBlock newDBlock(Block block, List locations) { - final DBlock db = new DBlock(block); + DBlock newDBlock(LocatedBlock lb, List locations, + ECSchema ecSchema) { + Block blk = lb.getBlock().getLocalBlock(); + DBlock db; + if (lb.isStriped()) { + LocatedStripedBlock lsb = (LocatedStripedBlock) lb; + byte[] indices = new byte[lsb.getBlockIndices().length]; + for (int i = 0; i < indices.length; i++) { + indices[i] = (byte) lsb.getBlockIndices()[i]; + } + db = new DBlockStriped(blk, indices, (short) ecSchema.getNumDataUnits()); + } else { + db = new DBlock(blk); + } for(MLocation ml : locations) { StorageGroup source = storages.getSource(ml); if (source != null) { @@ -358,9 +373,10 @@ private boolean processFile(String fullPath, HdfsLocatedFileStatus status) { LOG.warn("Failed to get the storage policy of file " + fullPath); return false; } - final List types = policy.chooseStorageTypes( + List types = policy.chooseStorageTypes( status.getReplication()); + final ECSchema ecSchema = status.getECSchema(); final LocatedBlocks locatedBlocks = status.getBlockLocations(); boolean hasRemaining = false; final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete(); @@ -371,10 +387,13 @@ private boolean processFile(String fullPath, HdfsLocatedFileStatus status) { continue; } LocatedBlock lb = lbs.get(i); + if (lb.isStriped()) { + types = policy.chooseStorageTypes((short) lb.getLocations().length); + } final StorageTypeDiff diff = new StorageTypeDiff(types, lb.getStorageTypes()); if (!diff.removeOverlap(true)) { - if (scheduleMoves4Block(diff, lb)) { + if (scheduleMoves4Block(diff, lb, ecSchema)) { hasRemaining |= (diff.existing.size() > 1 && diff.expected.size() > 1); } @@ -383,10 +402,13 @@ private boolean processFile(String fullPath, HdfsLocatedFileStatus status) { return hasRemaining; } - boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) { + boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb, + ECSchema ecSchema) { final List locations = MLocation.toLocations(lb); - Collections.shuffle(locations); - final DBlock db = newDBlock(lb.getBlock().getLocalBlock(), locations); + if (!(lb instanceof LocatedStripedBlock)) { + Collections.shuffle(locations); + } + final DBlock db = newDBlock(lb, locations, ecSchema); for (final StorageType t : diff.existing) { for (final MLocation ml : locations) { @@ -729,4 +751,4 @@ public static void main(String[] args) { System.exit(-1); } } -} \ No newline at end of file +} 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 a985dbdc8c0e3..0507faf1af128 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,7 @@ */ package org.apache.hadoop.hdfs.server.protocol; +import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; @@ -91,6 +92,30 @@ private StringBuilder appendString(int i, StringBuilder b) { } } + public static class StripedBlockWithLocations extends BlockWithLocations { + final byte[] indices; + final short dataBlockNum; + + public StripedBlockWithLocations(BlockWithLocations blk, byte[] indices, + short dataBlockNum) { + super(blk.getBlock(), blk.getDatanodeUuids(), blk.getStorageIDs(), + blk.getStorageTypes()); + Preconditions.checkArgument( + blk.getDatanodeUuids().length == indices.length); + this.indices = indices; + this.dataBlockNum = dataBlockNum; + + } + + public byte[] getIndices() { + return indices; + } + + public short getDataBlockNum() { + return dataBlockNum; + } + } + private final BlockWithLocations[] blocks; /** Constructor with one parameter */ 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 f64cf8f025eb2..e6db596bc932b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -532,6 +532,9 @@ message BlockWithLocationsProto { repeated string datanodeUuids = 2; // Datanodes with replicas of the block repeated string storageUuids = 3; // Storages with replicas of the block repeated StorageTypeProto storageTypes = 4; + + optional bytes indices = 5; + optional uint32 dataBlockNum = 6; } /** 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 6cd7003b3ee9b..db230e3266415 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 @@ -1964,4 +1964,19 @@ public static ExtendedBlock flushInternal(DFSStripedOutputStream out) out.flushInternal(); return out.getBlock(); } + + /** + * Verify that blocks in striped block group are on different nodes. + */ + public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, + int groupSize) { + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + HashSet locs = new HashSet<>(); + for (DatanodeInfo datanodeInfo : lb.getLocations()) { + locs.add(datanodeInfo); + } + assertEquals(groupSize, lb.getLocations().length); + assertEquals(groupSize, locs.size()); + } + } } 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 a0b203887e862..3675e63d66596 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 @@ -78,6 +78,7 @@ 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.BlocksWithLocations.StripedBlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; @@ -189,40 +190,58 @@ public void testConvertBlock() { assertEquals(b, b2); } - private static BlockWithLocations getBlockWithLocations(int bid) { + private static BlockWithLocations getBlockWithLocations( + int bid, boolean isStriped) { final String[] datanodeUuids = {"dn1", "dn2", "dn3"}; final String[] storageIDs = {"s1", "s2", "s3"}; final StorageType[] storageTypes = { StorageType.DISK, StorageType.DISK, StorageType.DISK}; - return new BlockWithLocations(new Block(bid, 0, 1), + final byte[] indices = {0, 1, 2}; + final short dataBlkNum = 6; + BlockWithLocations blkLocs = new BlockWithLocations(new Block(bid, 0, 1), datanodeUuids, storageIDs, storageTypes); + if (isStriped) { + blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum); + } + return blkLocs; } private void compare(BlockWithLocations locs1, BlockWithLocations locs2) { assertEquals(locs1.getBlock(), locs2.getBlock()); assertTrue(Arrays.equals(locs1.getStorageIDs(), locs2.getStorageIDs())); + if (locs1 instanceof StripedBlockWithLocations) { + assertTrue(Arrays.equals(((StripedBlockWithLocations) locs1).getIndices(), + ((StripedBlockWithLocations) locs2).getIndices())); + } } @Test public void testConvertBlockWithLocations() { - BlockWithLocations locs = getBlockWithLocations(1); - BlockWithLocationsProto locsProto = PBHelper.convert(locs); - BlockWithLocations locs2 = PBHelper.convert(locsProto); - compare(locs, locs2); + boolean[] testSuite = new boolean[]{false, true}; + for (int i = 0; i < testSuite.length; i++) { + BlockWithLocations locs = getBlockWithLocations(1, testSuite[i]); + BlockWithLocationsProto locsProto = PBHelper.convert(locs); + BlockWithLocations locs2 = PBHelper.convert(locsProto); + compare(locs, locs2); + } } @Test public void testConvertBlocksWithLocations() { - BlockWithLocations[] list = new BlockWithLocations[] { - getBlockWithLocations(1), getBlockWithLocations(2) }; - BlocksWithLocations locs = new BlocksWithLocations(list); - BlocksWithLocationsProto locsProto = PBHelper.convert(locs); - BlocksWithLocations locs2 = PBHelper.convert(locsProto); - BlockWithLocations[] blocks = locs.getBlocks(); - BlockWithLocations[] blocks2 = locs2.getBlocks(); - assertEquals(blocks.length, blocks2.length); - for (int i = 0; i < blocks.length; i++) { - compare(blocks[i], blocks2[i]); + boolean[] testSuite = new boolean[]{false, true}; + for (int i = 0; i < testSuite.length; i++) { + BlockWithLocations[] list = new BlockWithLocations[]{ + getBlockWithLocations(1, testSuite[i]), + getBlockWithLocations(2, testSuite[i])}; + BlocksWithLocations locs = new BlocksWithLocations(list); + BlocksWithLocationsProto locsProto = PBHelper.convert(locs); + BlocksWithLocations locs2 = PBHelper.convert(locsProto); + BlockWithLocations[] blocks = locs.getBlocks(); + BlockWithLocations[] blocks2 = locs2.getBlocks(); + assertEquals(blocks.length, blocks2.length); + for (int j = 0; j < blocks.length; j++) { + compare(blocks[j], blocks2[j]); + } } } 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 92d31d00af85a..f6475cd82b04d 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 @@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Tool; @@ -133,6 +134,21 @@ static void initConfWithRamDisk(Configuration conf, LazyPersistTestCase.initCacheManipulator(); } + int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + int groupSize = dataBlocks + parityBlocks; + private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock; + + static void initConfWithStripe(Configuration conf) { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE); + conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); + SimulatedFSDataset.setFactory(conf); + conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); + conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L); + } + /* create a file with a length of fileLen */ static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen, short replicationFactor, int nnIndex) @@ -1452,6 +1468,66 @@ public void testManyBalancerSimultaneously() throws Exception { } } + @Test(timeout = 100000) + public void testBalancerWithStripedFile() throws Exception { + Configuration conf = new Configuration(); + initConfWithStripe(conf); + int numOfDatanodes = dataBlocks + parityBlocks + 2; + int numOfRacks = dataBlocks; + long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE; + long[] capacities = new long[numOfDatanodes]; + for (int i = 0; i < capacities.length; i++) { + capacities[i] = capacity; + } + String[] racks = new String[numOfDatanodes]; + for (int i = 0; i < numOfDatanodes; i++) { + racks[i] = "/rack" + (i % numOfRacks); + } + cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(numOfDatanodes) + .racks(racks) + .simulatedCapacities(capacities) + .build(); + + try { + cluster.waitActive(); + client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(), + ClientProtocol.class).getProxy(); + client.createErasureCodingZone("/", null, 0); + + long totalCapacity = sum(capacities); + + // fill up the cluster with 30% data. It'll be 45% full plus parity. + long fileLen = totalCapacity * 3 / 10; + long totalUsedSpace = fileLen * (dataBlocks + parityBlocks) / dataBlocks; + FileSystem fs = cluster.getFileSystem(0); + DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, r.nextLong()); + + // verify locations of striped blocks + LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen); + DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); + + // add one datanode + String newRack = "/rack" + (++numOfRacks); + cluster.startDataNodes(conf, 1, true, null, + new String[]{newRack}, null, new long[]{capacity}); + totalCapacity += capacity; + cluster.triggerHeartbeats(); + + // run balancer and validate results + Balancer.Parameters p = Balancer.Parameters.DEFAULT; + Collection namenodes = DFSUtil.getNsServiceRpcUris(conf); + runBalancer(conf, totalUsedSpace, totalCapacity, p, 0); + + // verify locations of striped blocks + locatedBlocks = client.getBlockLocations(fileName, 0, fileLen); + DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); + + } finally { + cluster.shutdown(); + } + } + /** * @param args */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index f4bedabf82a08..74f09fdabd1bc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DFSTestUtil; @@ -34,10 +35,16 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; +import org.apache.hadoop.hdfs.NameNodeProxies; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.balancer.Dispatcher; import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock; import org.apache.hadoop.hdfs.server.balancer.ExitStatus; import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.mover.Mover.MLocation; import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; import org.apache.hadoop.test.GenericTestUtils; @@ -83,7 +90,7 @@ public void testScheduleSameBlock() throws IOException { final LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0); final List locations = MLocation.toLocations(lb); final MLocation ml = locations.get(0); - final DBlock db = mover.newDBlock(lb.getBlock().getLocalBlock(), locations); + final DBlock db = mover.newDBlock(lb, locations, null); final List storageTypes = new ArrayList( Arrays.asList(StorageType.DEFAULT, StorageType.DEFAULT)); @@ -361,4 +368,119 @@ public void testMoverFailedRetry() throws Exception { cluster.shutdown(); } } + + int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock; + + static void initConfWithStripe(Configuration conf) { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE); + conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); + conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); + Dispatcher.setBlockMoveWaitTime(3000L); + } + + @Test(timeout = 300000) + public void testMoverWithStripedFile() throws Exception { + final Configuration conf = new HdfsConfiguration(); + initConfWithStripe(conf); + + // start 10 datanodes + int numOfDatanodes =10; + int storagesPerDatanode=2; + long capacity = 10 * DEFAULT_STRIPE_BLOCK_SIZE; + long[][] capacities = new long[numOfDatanodes][storagesPerDatanode]; + for (int i = 0; i < numOfDatanodes; i++) { + for(int j=0;j Date: Thu, 4 Jun 2015 14:41:38 +0800 Subject: [PATCH 149/212] HDFS-8328. Follow-on to update decode for DataNode striped blocks reconstruction. (yliu) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 +- .../erasurecode/ErasureCodingWorker.java | 421 ++++++++++-------- .../hadoop/hdfs/TestRecoverStripedFile.java | 65 ++- 4 files changed, 305 insertions(+), 186 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 511ebecc20ae9..a160520fa1614 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -280,3 +280,6 @@ HDFS-7621. Erasure Coding: update the Balancer/Mover data migration logic. (Walter Su via zhz) + + HDFS-8328. Follow-on to update decode for DataNode striped blocks + reconstruction. (yliu) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 8e6b9f0b936a3..77dd1fdbe883f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -373,7 +373,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_DATANODE_STRIPED_READ_THREADS_KEY = "dfs.datanode.stripedread.threads"; public static final int DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20; public static final String DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size"; - public static final int DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 256 * 1024; + public static final int DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024; public static final String DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY = "dfs.datanode.stripedread.threshold.millis"; public static final int DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s public static final String DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 00cf0fd5e2c3a..6f3857feee9ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -70,9 +70,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Daemon; @@ -80,6 +78,8 @@ import com.google.common.base.Preconditions; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode; + /** * ErasureCodingWorker handles the erasure coding recovery work commands. These * commands would be issued from Namenode as part of Datanode's heart beat @@ -110,10 +110,6 @@ public ErasureCodingWorker(Configuration conf, DataNode datanode) { DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT); } - - private RawErasureEncoder newEncoder(int numDataUnits, int numParityUnits) { - return new RSRawEncoder(numDataUnits, numParityUnits); - } private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) { return new RSRawDecoder(numDataUnits, numParityUnits); @@ -221,14 +217,14 @@ private class ReconstructAndTransferBlock implements Runnable { private final int parityBlkNum; private final int cellSize; - private RawErasureEncoder encoder; private RawErasureDecoder decoder; // Striped read buffer size private int bufferSize; private final ExtendedBlock blockGroup; - // position in striped block + private final int minRequiredSources; + // position in striped internal block private long positionInBlock; // sources @@ -237,6 +233,10 @@ private class ReconstructAndTransferBlock implements Runnable { private final List stripedReaders; + // The buffers and indices for striped blocks whose length is 0 + private ByteBuffer[] zeroStripeBuffers; + private short[] zeroStripeIndices; + // targets private final DatanodeInfo[] targets; private final StorageType[] targetStorageTypes; @@ -272,21 +272,32 @@ private class ReconstructAndTransferBlock implements Runnable { cellSize = recoveryInfo.getCellSize(); blockGroup = recoveryInfo.getExtendedBlock(); + final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1); + minRequiredSources = Math.min(cellsNum, dataBlkNum); liveIndices = recoveryInfo.getLiveBlockIndices(); sources = recoveryInfo.getSourceDnInfos(); stripedReaders = new ArrayList<>(sources.length); - Preconditions.checkArgument(liveIndices.length >= dataBlkNum, + Preconditions.checkArgument(liveIndices.length >= minRequiredSources, "No enough live striped blocks."); Preconditions.checkArgument(liveIndices.length == sources.length, "liveBlockIndices and source dns should match"); + if (minRequiredSources < dataBlkNum) { + zeroStripeBuffers = + new ByteBuffer[dataBlkNum - minRequiredSources]; + zeroStripeIndices = new short[dataBlkNum - minRequiredSources]; + } + targets = recoveryInfo.getTargetDnInfos(); targetStorageTypes = recoveryInfo.getTargetStorageTypes(); targetIndices = new short[targets.length]; targetBuffers = new ByteBuffer[targets.length]; + Preconditions.checkArgument(targetIndices.length <= parityBlkNum, + "Too much missed striped blocks."); + targetSockets = new Socket[targets.length]; targetOutputStreams = new DataOutputStream[targets.length]; targetInputStreams = new DataInputStream[targets.length]; @@ -303,6 +314,10 @@ private class ReconstructAndTransferBlock implements Runnable { cachingStrategy = CachingStrategy.newDefaultStrategy(); } + private ByteBuffer allocateBuffer(int length) { + return ByteBuffer.allocate(length); + } + private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) { return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize, dataBlkNum, i); @@ -313,37 +328,67 @@ private long getBlockLen(ExtendedBlock blockGroup, int i) { cellSize, dataBlkNum, i); } + /** + * StripedReader is used to read from one source DN, it contains a block + * reader, buffer and striped block index. + * Only allocate StripedReader once for one source, and the StripedReader + * has the same array order with sources. Typically we only need to allocate + * minimum number (minRequiredSources) of StripedReader, and allocate + * new for new source DN if some existing DN invalid or slow. + * If some source DN is corrupt, set the corresponding blockReader to + * null and will never read from it again. + * + * @param i the array index of sources + * @param offsetInBlock offset for the internal block + * @return StripedReader + */ + private StripedReader addStripedReader(int i, long offsetInBlock) { + StripedReader reader = new StripedReader(liveIndices[i]); + stripedReaders.add(reader); + + BlockReader blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]); + if (blockReader != null) { + initChecksumAndBufferSizeIfNeeded(blockReader); + reader.blockReader = blockReader; + } + reader.buffer = allocateBuffer(bufferSize); + return reader; + } + @Override public void run() { datanode.incrementXmitsInProgress(); try { - // Store the indices of successfully read source - // This will be updated after doing real read. - int[] success = new int[dataBlkNum]; + // Store the array indices of source DNs we have read successfully. + // In each iteration of read, the success list may be updated if + // some source DN is corrupted or slow. And use the updated success + // list of DNs for next iteration read. + int[] success = new int[minRequiredSources]; int nsuccess = 0; - for (int i = 0; i < sources.length && nsuccess < dataBlkNum; i++) { - StripedReader reader = new StripedReader(liveIndices[i]); - stripedReaders.add(reader); - - BlockReader blockReader = newBlockReader( - getBlock(blockGroup, liveIndices[i]), 0, sources[i]); - if (blockReader != null) { - initChecksumAndBufferSizeIfNeeded(blockReader); - reader.blockReader = blockReader; - reader.buffer = ByteBuffer.allocate(bufferSize); + for (int i = 0; + i < sources.length && nsuccess < minRequiredSources; i++) { + StripedReader reader = addStripedReader(i, 0); + if (reader.blockReader != null) { success[nsuccess++] = i; } } - if (nsuccess < dataBlkNum) { + if (nsuccess < minRequiredSources) { String error = "Can't find minimum sources required by " + "recovery, block id: " + blockGroup.getBlockId(); throw new IOException(error); } + if (zeroStripeBuffers != null) { + for (int i = 0; i < zeroStripeBuffers.length; i++) { + zeroStripeBuffers[i] = allocateBuffer(bufferSize); + } + } + for (int i = 0; i < targets.length; i++) { - targetBuffers[i] = ByteBuffer.allocate(bufferSize); + targetBuffers[i] = allocateBuffer(bufferSize); } checksumSize = checksum.getChecksumSize(); @@ -356,7 +401,9 @@ public void run() { packetBuf = new byte[maxPacketSize]; checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)]; - // Store whether the target is success + // targetsStatus store whether some target is success, it will record + // any failed target once, if some target failed (invalid DN or transfer + // failed), will not transfer data to it any more. boolean[] targetsStatus = new boolean[targets.length]; if (initTargetStreams(targetsStatus) == 0) { String error = "All targets are failed."; @@ -367,16 +414,11 @@ public void run() { while (positionInBlock < firstStripedBlockLength) { int toRead = Math.min( bufferSize, (int)(firstStripedBlockLength - positionInBlock)); - // step1: read minimum striped buffer size data required by recovery. - nsuccess = readMinimumStripedData4Recovery(success); - - if (nsuccess < dataBlkNum) { - String error = "Can't read data from minimum number of sources " - + "required by recovery, block id: " + blockGroup.getBlockId(); - throw new IOException(error); - } + // step1: read from minimum source DNs required for reconstruction. + // The returned success list is the source DNs we do real read from + success = readMinimumStripedData4Recovery(success); - // step2: encode/decode to recover targets + // step2: decode to reconstruct targets long remaining = firstStripedBlockLength - positionInBlock; int toRecoverLen = remaining < bufferSize ? (int)remaining : bufferSize; @@ -426,65 +468,97 @@ private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) { } } - // assume liveIndices is not ordered. private void getTargetIndices() { BitSet bitset = new BitSet(dataBlkNum + parityBlkNum); for (int i = 0; i < sources.length; i++) { bitset.set(liveIndices[i]); } int m = 0; - for (int i = 0; i < dataBlkNum + parityBlkNum && m < targets.length; i++) { + int k = 0; + for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { if (!bitset.get(i)) { - targetIndices[m++] = (short)i; + if (getBlockLen(blockGroup, i) > 0) { + if (m < targets.length) { + targetIndices[m++] = (short)i; + } + } else { + zeroStripeIndices[k++] = (short)i; + } } } } + private long getReadLength(int index) { + long blockLen = getBlockLen(blockGroup, index); + long remaining = blockLen - positionInBlock; + return remaining > bufferSize ? bufferSize : remaining; + } + /** - * Read minimum striped buffer size data required by recovery. - * success list will be updated after read. + * Read from minimum source DNs required for reconstruction in the iteration. + * First try the success list which we think they are the best DNs + * If source DN is corrupt or slow, try to read some other source DN, + * and will update the success list. + * + * Remember the updated success list and return it for following + * operations and next iteration read. * - * Initially we only read from dataBlkNum sources, - * if timeout or failure for some source, we will try to schedule - * read from a new source. + * @param success the initial success list of source DNs we think best + * @return updated success list of source DNs we do real read + * @throws IOException */ - private int readMinimumStripedData4Recovery(int[] success) { - + private int[] readMinimumStripedData4Recovery(final int[] success) + throws IOException { + int nsuccess = 0; + int[] newSuccess = new int[minRequiredSources]; BitSet used = new BitSet(sources.length); - for (int i = 0; i < dataBlkNum; i++) { + /* + * Read from minimum source DNs required, the success list contains + * source DNs which we think best. + */ + for (int i = 0; i < minRequiredSources; i++) { StripedReader reader = stripedReaders.get(success[i]); - Callable readCallable = readFromBlock( - reader.blockReader, reader.buffer); - Future f = readService.submit(readCallable); - futures.put(f, success[i]); + if (getReadLength(liveIndices[success[i]]) > 0) { + Callable readCallable = readFromBlock( + reader.blockReader, reader.buffer); + Future f = readService.submit(readCallable); + futures.put(f, success[i]); + } else { + // If the read length is 0, we don't need to do real read + reader.buffer.position(0); + newSuccess[nsuccess++] = success[i]; + } used.set(success[i]); } - int nsuccess = 0; while (!futures.isEmpty()) { try { StripingChunkReadResult result = StripedBlockUtil.getNextCompletedStripedRead( readService, futures, STRIPED_READ_THRESHOLD_MILLIS); + int resultIndex = -1; if (result.state == StripingChunkReadResult.SUCCESSFUL) { - success[nsuccess++] = result.index; - if (nsuccess >= dataBlkNum) { - // cancel remaining reads if we read successfully from minimum - // number of sources required for recovery. - cancelReads(futures.keySet()); - futures.clear(); - break; - } + resultIndex = result.index; } else if (result.state == StripingChunkReadResult.FAILED) { - // If read failed for some source, we should not use it anymore - // and schedule read from a new source. + // If read failed for some source DN, we should not use it anymore + // and schedule read from another source DN. StripedReader failedReader = stripedReaders.get(result.index); closeBlockReader(failedReader.blockReader); failedReader.blockReader = null; - scheduleNewRead(used); + resultIndex = scheduleNewRead(used); } else if (result.state == StripingChunkReadResult.TIMEOUT) { // If timeout, we also schedule a new read. - scheduleNewRead(used); + resultIndex = scheduleNewRead(used); + } + if (resultIndex >= 0) { + newSuccess[nsuccess++] = resultIndex; + if (nsuccess >= minRequiredSources) { + // cancel remaining reads if we read successfully from minimum + // number of source DNs required by reconstruction. + cancelReads(futures.keySet()); + futures.clear(); + break; + } } } catch (InterruptedException e) { LOG.info("Read data interrupted.", e); @@ -492,19 +566,13 @@ private int readMinimumStripedData4Recovery(int[] success) { } } - return nsuccess; - } - - /** - * Return true if need to do encoding to recovery missed striped block. - */ - private boolean shouldEncode(int[] success) { - for (int i = 0; i < success.length; i++) { - if (stripedReaders.get(success[i]).index >= dataBlkNum) { - return false; - } + if (nsuccess < minRequiredSources) { + String error = "Can't read data from minimum number of sources " + + "required by reconstruction, block id: " + blockGroup.getBlockId(); + throw new IOException(error); } - return true; + + return newSuccess; } private void paddingBufferToLen(ByteBuffer buffer, int len) { @@ -514,13 +582,6 @@ private void paddingBufferToLen(ByteBuffer buffer, int len) { } } - // Initialize encoder - private void initEncoderIfNecessary() { - if (encoder == null) { - encoder = newEncoder(dataBlkNum, parityBlkNum); - } - } - // Initialize decoder private void initDecoderIfNecessary() { if (decoder == null) { @@ -528,119 +589,119 @@ private void initDecoderIfNecessary() { } } + private int[] getErasedIndices(boolean[] targetsStatus) { + int[] result = new int[targets.length]; + int m = 0; + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + result[m++] = convertIndex4Decode(targetIndices[i], + dataBlkNum, parityBlkNum); + } + } + return Arrays.copyOf(result, m); + } + private void recoverTargets(int[] success, boolean[] targetsStatus, int toRecoverLen) { - if (shouldEncode(success)) { - initEncoderIfNecessary(); - ByteBuffer[] dataBuffers = new ByteBuffer[dataBlkNum]; - ByteBuffer[] parityBuffers = new ByteBuffer[parityBlkNum]; - for (int i = 0; i < dataBlkNum; i++) { - StripedReader reader = stripedReaders.get(i); - ByteBuffer buffer = reader.buffer; - paddingBufferToLen(buffer, toRecoverLen); - dataBuffers[i] = (ByteBuffer)buffer.flip(); - } - for (int i = dataBlkNum; i < stripedReaders.size(); i++) { - StripedReader reader = stripedReaders.get(i); - parityBuffers[reader.index - dataBlkNum] = cleanBuffer(reader.buffer); - } - for (int i = 0; i < targets.length; i++) { - parityBuffers[targetIndices[i] - dataBlkNum] = targetBuffers[i]; - } - for (int i = 0; i < parityBlkNum; i++) { - if (parityBuffers[i] == null) { - parityBuffers[i] = ByteBuffer.allocate(toRecoverLen); - } else { - parityBuffers[i].limit(toRecoverLen); - } - } - encoder.encode(dataBuffers, parityBuffers); - } else { - /////////// TODO: wait for HADOOP-11847 ///////////// - ////////// The current decode method always try to decode parityBlkNum number of data blocks. //////////// - initDecoderIfNecessary(); - ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum]; - for (int i = 0; i < success.length; i++) { - StripedReader reader = stripedReaders.get(success[i]); - ByteBuffer buffer = reader.buffer; + initDecoderIfNecessary(); + ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum]; + for (int i = 0; i < success.length; i++) { + StripedReader reader = stripedReaders.get(success[i]); + ByteBuffer buffer = reader.buffer; + paddingBufferToLen(buffer, toRecoverLen); + inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = + (ByteBuffer)buffer.flip(); + } + if (success.length < dataBlkNum) { + for (int i = 0; i < zeroStripeBuffers.length; i++) { + ByteBuffer buffer = zeroStripeBuffers[i]; paddingBufferToLen(buffer, toRecoverLen); - int index = reader.index < dataBlkNum ? - reader.index + parityBlkNum : reader.index - dataBlkNum; + int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum, + parityBlkNum); inputs[index] = (ByteBuffer)buffer.flip(); } - int[] indices4Decode = new int[parityBlkNum]; - int m = 0; - for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { - if (inputs[i] == null) { - inputs[i] = ByteBuffer.allocate(toRecoverLen); - indices4Decode[m++] = i; - } - } - ByteBuffer[] outputs = new ByteBuffer[parityBlkNum]; - m = 0; - // targetIndices is subset of indices4Decode - for (int i = 0; i < parityBlkNum; i++) { - if (m < targetIndices.length && - (indices4Decode[i] - parityBlkNum) == targetIndices[m]) { - outputs[i] = targetBuffers[m++]; - outputs[i].limit(toRecoverLen); - } else { - outputs[i] = ByteBuffer.allocate(toRecoverLen); - } + } + int[] erasedIndices = getErasedIndices(targetsStatus); + ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length]; + int m = 0; + for (int i = 0; i < targetBuffers.length; i++) { + if (targetsStatus[i]) { + outputs[m++] = targetBuffers[i]; + outputs[i].limit(toRecoverLen); } - - decoder.decode(inputs, indices4Decode, outputs); - - for (int i = 0; i < targets.length; i++) { - if (targetsStatus[i]) { - long blockLen = getBlockLen(blockGroup, targetIndices[i]); - long remaining = blockLen - positionInBlock; - if (remaining < 0) { - targetBuffers[i].limit(0); - } else if (remaining < toRecoverLen) { - targetBuffers[i].limit((int)remaining); - } + } + decoder.decode(inputs, erasedIndices, outputs); + + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + long blockLen = getBlockLen(blockGroup, targetIndices[i]); + long remaining = blockLen - positionInBlock; + if (remaining < 0) { + targetBuffers[i].limit(0); + } else if (remaining < toRecoverLen) { + targetBuffers[i].limit((int)remaining); } } } } - /** - * Schedule read from a new source, we first try un-initial source, - * then try un-used source in this round and bypass failed source. + /** + * Schedule a read from some new source DN if some DN is corrupted + * or slow, this is called from the read iteration. + * Initially we may only have minRequiredSources number of + * StripedReader. + * If the position is at the end of target block, don't need to do + * real read, and return the array index of source DN, otherwise -1. + * + * @param used the used source DNs in this iteration. + * @return the array index of source DN if don't need to do real read. */ - private void scheduleNewRead(BitSet used) { + private int scheduleNewRead(BitSet used) { StripedReader reader = null; + // step1: initially we may only have minRequiredSources + // number of StripedReader, and there may be some source DNs we never + // read before, so will try to create StripedReader for one new source DN + // and try to read from it. If found, go to step 3. int m = stripedReaders.size(); - while (m < sources.length && reader == null) { - reader = new StripedReader(liveIndices[m]); - BlockReader blockReader = newBlockReader( - getBlock(blockGroup, liveIndices[m]), positionInBlock, sources[m]); - stripedReaders.add(reader); - if (blockReader != null) { - assert blockReader.getDataChecksum().equals(checksum); - reader.blockReader = blockReader; - reader.buffer = ByteBuffer.allocate(bufferSize); + while (reader == null && m < sources.length) { + reader = addStripedReader(m, positionInBlock); + if (getReadLength(liveIndices[m]) > 0) { + if (reader.blockReader == null) { + reader = null; + m++; + } } else { - m++; - reader = null; + used.set(m); + return m; } } + // step2: if there is no new source DN we can use, try to find a source + // DN we ever read from but because some reason, e.g., slow, it + // is not in the success DN list at the begin of this iteration, so + // we have not tried it in this iteration. Now we have a chance to + // revisit it again. for (int i = 0; reader == null && i < stripedReaders.size(); i++) { - StripedReader r = stripedReaders.get(i); - if (r.blockReader != null && !used.get(i)) { - closeBlockReader(r.blockReader); - r.blockReader = newBlockReader( - getBlock(blockGroup, liveIndices[i]), positionInBlock, - sources[i]); - if (r.blockReader != null) { - m = i; - reader = r; + if (!used.get(i)) { + StripedReader r = stripedReaders.get(i); + if (getReadLength(liveIndices[i]) > 0) { + closeBlockReader(r.blockReader); + r.blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[i]), positionInBlock, + sources[i]); + if (r.blockReader != null) { + m = i; + reader = r; + } + } else { + used.set(i); + r.buffer.position(0); + return i; } } } + // step3: schedule if find a correct source DN and need to do real read. if (reader != null) { Callable readCallable = readFromBlock( reader.blockReader, reader.buffer); @@ -648,6 +709,8 @@ private void scheduleNewRead(BitSet used) { futures.put(f, m); used.set(m); } + + return -1; } // cancel all reads. @@ -708,7 +771,10 @@ private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) { } private BlockReader newBlockReader(final ExtendedBlock block, - long startOffset, DatanodeInfo dnInfo) { + long offsetInBlock, DatanodeInfo dnInfo) { + if (offsetInBlock >= block.getNumBytes()) { + return null; + } try { InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo); Token blockToken = datanode.getBlockAccessToken( @@ -720,7 +786,8 @@ private BlockReader newBlockReader(final ExtendedBlock block, * requires config for domain-socket in UNIX or legacy config in Windows. */ return RemoteBlockReader2.newBlockReader( - "dummy", block, blockToken, startOffset, block.getNumBytes(), true, + "dummy", block, blockToken, offsetInBlock, + block.getNumBytes() - offsetInBlock, true, "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo, null, cachingStrategy); } catch (IOException e) { @@ -808,6 +875,12 @@ private void clearBuffers() { } } + if (zeroStripeBuffers != null) { + for (int i = 0; i < zeroStripeBuffers.length; i++) { + zeroStripeBuffers[i].clear(); + } + } + for (int i = 0; i < targetBuffers.length; i++) { if (targetBuffers[i] != null) { cleanBuffer(targetBuffers[i]); @@ -903,7 +976,7 @@ private int initTargetStreams(boolean[] targetsStatus) { } private static class StripedReader { - private final short index; + private final short index; // internal block index private BlockReader blockReader; private ByteBuffer buffer; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index 9e44761d20688..9285fd73feb77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -100,29 +100,69 @@ public void testRecoverOneParityBlock() throws Exception { } @Test(timeout = 120000) - public void testRecoverThreeParityBlocks() throws Exception { + public void testRecoverOneParityBlock1() throws Exception { + int fileLen = cellSize + cellSize/10; + assertFileBlocksRecovery("/testRecoverOneParityBlock1", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock2() throws Exception { + int fileLen = 1; + assertFileBlocksRecovery("/testRecoverOneParityBlock2", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock3() throws Exception { int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverOneParityBlock3", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverThreeParityBlocks() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen, 0, 3); } @Test(timeout = 120000) public void testRecoverThreeDataBlocks() throws Exception { - int fileLen = 3 * blockSize + blockSize/10; + int fileLen = 10 * blockSize + blockSize/10; assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen, 1, 3); } + @Test(timeout = 120000) + public void testRecoverThreeDataBlocks1() throws Exception { + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverThreeDataBlocks1", fileLen, 1, 3); + } + @Test(timeout = 120000) public void testRecoverOneDataBlock() throws Exception { - ////TODO: TODO: wait for HADOOP-11847 - //int fileLen = 10 * blockSize + blockSize/10; - //assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1); + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneDataBlock1() throws Exception { + int fileLen = cellSize + cellSize/10; + assertFileBlocksRecovery("/testRecoverOneDataBlock1", fileLen, 1, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneDataBlock2() throws Exception { + int fileLen = 1; + assertFileBlocksRecovery("/testRecoverOneDataBlock2", fileLen, 1, 1); } @Test(timeout = 120000) public void testRecoverAnyBlocks() throws Exception { - ////TODO: TODO: wait for HADOOP-11847 - //int fileLen = 3 * blockSize + blockSize/10; - //assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2); + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2); + } + + @Test(timeout = 120000) + public void testRecoverAnyBlocks1() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverAnyBlocks1", fileLen, 2, 3); } /** @@ -203,6 +243,9 @@ private void assertFileBlocksRecovery(String fileName, int fileLen, replicaContents[i] = readReplica(replicas[i]); } + int cellsNum = (fileLen - 1) / cellSize + 1; + int groupSize = Math.min(cellsNum, dataBlkNum) + parityBlkNum; + try { DatanodeID[] dnIDs = new DatanodeID[toRecoverBlockNum]; for (int i = 0; i < toRecoverBlockNum; i++) { @@ -216,7 +259,6 @@ private void assertFileBlocksRecovery(String fileName, int fileLen, dnIDs[i] = dn.getDatanodeId(); } setDataNodesDead(dnIDs); - // Check the locatedBlocks of the file again locatedBlocks = getLocatedBlocks(file); @@ -232,7 +274,7 @@ private void assertFileBlocksRecovery(String fileName, int fileLen, } } - waitForRecoveryFinished(file); + waitForRecoveryFinished(file, groupSize); targetDNs = sortTargetsByReplicas(blocks, targetDNs); @@ -319,7 +361,8 @@ private byte[] readReplica(File replica) throws IOException { } } - private LocatedBlocks waitForRecoveryFinished(Path file) throws Exception { + private LocatedBlocks waitForRecoveryFinished(Path file, int groupSize) + throws Exception { final int ATTEMPTS = 60; for (int i = 0; i < ATTEMPTS; i++) { LocatedBlocks locatedBlocks = getLocatedBlocks(file); From c0929ab3c2de76e2d514b7dec11354fea40ea089 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 4 Jun 2015 18:16:41 -0700 Subject: [PATCH 150/212] HDFS-8319. Erasure Coding: support decoding for stateful read. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSInputStream.java | 2 +- .../hadoop/hdfs/DFSStripedInputStream.java | 595 ++++++++++++------ .../hadoop/hdfs/util/StripedBlockUtil.java | 215 +++++-- .../hadoop/hdfs/StripedFileTestUtil.java | 2 +- .../hdfs/TestReadStripedFileWithDecoding.java | 41 +- .../hadoop/hdfs/TestWriteReadStripedFile.java | 69 +- .../hdfs/util/TestStripedBlockUtil.java | 12 +- 8 files changed, 649 insertions(+), 290 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index a160520fa1614..9b05d2bc7013b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -283,3 +283,6 @@ HDFS-8328. Follow-on to update decode for DataNode striped blocks reconstruction. (yliu) + + HDFS-8319. Erasure Coding: support decoding for stateful read. + (Jing Zhao via zhz) 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 6102edfe92a44..1b5705a8b23d7 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 @@ -1639,7 +1639,7 @@ public synchronized boolean seekToNewSource(long targetPos) throws IOException { /** */ @Override - public synchronized long getPos() throws IOException { + public synchronized long getPos() { return pos; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 2e26cca094afa..bf99f17d7fd0e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -30,12 +30,13 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.ByteBufferPool; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.finalizeDecodeInputs; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getStartOffsetsForInternalBlocks; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; @@ -55,6 +56,7 @@ import java.io.InterruptedIOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.EnumSet; import java.util.Set; import java.util.Collection; @@ -63,8 +65,6 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.CancellationException; import java.util.concurrent.Callable; import java.util.concurrent.Future; @@ -113,11 +113,19 @@ boolean include(long pos) { } private final BlockReader[] blockReaders; + /** + * when initializing block readers, their starting offsets are set to the same + * number: the smallest internal block offsets among all the readers. This is + * because it is possible that for some internal blocks we have to read + * "backwards" for decoding purpose. We thus use this offset array to track + * offsets for all the block readers so that we can skip data if necessary. + */ + private final long[] blockReaderOffsets; private final DatanodeInfo[] currentNodes; private final int cellSize; private final short dataBlkNum; private final short parityBlkNum; - private final short groupSize; + private final int groupSize; /** the buffer for a complete stripe */ private ByteBuffer curStripeBuf; private final ECSchema schema; @@ -128,7 +136,8 @@ boolean include(long pos) { * block group */ private StripeRange curStripeRange; - private final CompletionService readingService; + private final CompletionService readingService; + private ReaderRetryPolicy retry; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECSchema schema, int cellSize) throws IOException { @@ -139,8 +148,9 @@ boolean include(long pos) { this.cellSize = cellSize; dataBlkNum = (short) schema.getNumDataUnits(); parityBlkNum = (short) schema.getNumParityUnits(); - groupSize = dataBlkNum; + groupSize = dataBlkNum + parityBlkNum; blockReaders = new BlockReader[groupSize]; + blockReaderOffsets = new long[groupSize]; currentNodes = new DatanodeInfo[groupSize]; curStripeRange = new StripeRange(0, 0); readingService = @@ -197,20 +207,21 @@ private synchronized void blockSeekTo(long target) throws IOException { // The purpose is to get start offset into each block. long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema, cellSize, targetBlockGroup, offsetIntoBlockGroup); - Preconditions.checkNotNull(offsetsForInternalBlocks); + Preconditions.checkState( + offsetsForInternalBlocks.length == dataBlkNum + parityBlkNum); + long minOffset = offsetsForInternalBlocks[dataBlkNum]; - final ReaderRetryPolicy retry = new ReaderRetryPolicy(); - for (int i = 0; i < groupSize; i++) { + retry = new ReaderRetryPolicy(); + for (int i = 0; i < dataBlkNum; i++) { LocatedBlock targetBlock = targetBlocks[i]; if (targetBlock != null) { - long offsetInBlock = offsetsForInternalBlocks[i] < 0 ? - 0 : offsetsForInternalBlocks[i]; DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); if (retval != null) { currentNodes[i] = retval.info; blockReaders[i] = getBlockReaderWithRetry(targetBlock, - offsetInBlock, targetBlock.getBlockSize() - offsetInBlock, + minOffset, targetBlock.getBlockSize() - minOffset, retval.addr, retval.storageType, retval.info, target, retry); + blockReaderOffsets[i] = minOffset; } } } @@ -260,19 +271,24 @@ protected void closeCurrentBlockReaders() { return; } for (int i = 0; i < groupSize; i++) { - if (blockReaders[i] != null) { - try { - blockReaders[i].close(); - } catch (IOException e) { - DFSClient.LOG.error("error closing blockReader", e); - } - blockReaders[i] = null; - } + closeReader(i); currentNodes[i] = null; } blockEnd = -1; } + private void closeReader(int index) { + if (blockReaders[index] != null) { + try { + blockReaders[index].close(); + } catch (IOException e) { + DFSClient.LOG.error("error closing blockReader " + index, e); + } + blockReaders[index] = null; + } + blockReaderOffsets[index] = 0; + } + private long getOffsetInBlockGroup() { return getOffsetInBlockGroup(pos); } @@ -300,54 +316,81 @@ private void readOneStripe( curStripeRange = new StripeRange(offsetInBlockGroup, stripeLimit - stripeBufOffset); - final int startCell = stripeBufOffset / cellSize; - final int numCell = (stripeLimit - 1) / cellSize + 1; - - // read the whole stripe in parallel - Map, Integer> futures = new HashMap<>(); - for (int i = startCell; i < numCell; i++) { - int bufPos = i == startCell ? stripeBufOffset : cellSize * i; - curStripeBuf.position(bufPos); - curStripeBuf.limit(Math.min(cellSize * (i + 1), stripeLimit)); - ByteBuffer buf = curStripeBuf.slice(); - ByteBufferStrategy strategy = new ByteBufferStrategy(buf); - final int targetLength = buf.remaining(); - Callable readCallable = readCell(blockReaders[i], - currentNodes[i], strategy, targetLength, corruptedBlockMap); - Future request = readingService.submit(readCallable); - futures.put(request, i); - } - while (!futures.isEmpty()) { - try { - waitNextCompletion(readingService, futures); - // TODO: decode and record bad reader if necessary - } catch (InterruptedException ignored) { - // ignore and retry - } + LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock; + AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(schema, cellSize, + blockGroup, offsetInBlockGroup, + offsetInBlockGroup + curStripeRange.length - 1, curStripeBuf); + // TODO handle null elements in blks (e.g., NN does not know locations for + // all the internal blocks) + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( + blockGroup, cellSize, dataBlkNum, parityBlkNum); + // read the whole stripe + for (AlignedStripe stripe : stripes) { + // Parse group to get chosen DN location + StripeReader sreader = new StatefulStripeReader(readingService, stripe, + blks); + sreader.readStripe(blks, corruptedBlockMap); } + curStripeBuf.position(stripeBufOffset); + curStripeBuf.limit(stripeLimit); } - private Callable readCell(final BlockReader reader, - final DatanodeInfo datanode, final ByteBufferStrategy strategy, + private Callable readCell(final BlockReader reader, + final DatanodeInfo datanode, final long currentReaderOffset, + final long targetReaderOffset, final ByteBufferStrategy strategy, final int targetLength, final Map> corruptedBlockMap) { - return new Callable() { + return new Callable() { @Override - public Integer call() throws Exception { + public Void call() throws Exception { + // reader can be null if getBlockReaderWithRetry failed or + // the reader hit exception before + if (reader == null) { + throw new IOException("The BlockReader is null. " + + "The BlockReader creation failed or the reader hit exception."); + } + Preconditions.checkState(currentReaderOffset <= targetReaderOffset); + if (currentReaderOffset < targetReaderOffset) { + long skipped = reader.skip(targetReaderOffset - currentReaderOffset); + Preconditions.checkState( + skipped == targetReaderOffset - currentReaderOffset); + } int result = 0; while (result < targetLength) { - int ret = readBuffer(reader, datanode, strategy, corruptedBlockMap); + int ret = readToBuffer(reader, datanode, strategy, corruptedBlockMap); if (ret < 0) { throw new IOException("Unexpected EOS from the reader"); } result += ret; } updateReadStatistics(readStatistics, targetLength, reader); - return result; + return null; } }; } + private int readToBuffer(BlockReader blockReader, + DatanodeInfo currentNode, ByteBufferStrategy readerStrategy, + Map> corruptedBlockMap) + throws IOException { + try { + return readerStrategy.doRead(blockReader, 0, 0); + } catch (ChecksumException ce) { + DFSClient.LOG.warn("Found Checksum error for " + + getCurrentBlock() + " from " + currentNode + + " at " + ce.getPos()); + // we want to remember which block replicas we have tried + addIntoCorruptedBlockMap(getCurrentBlock(), currentNode, + corruptedBlockMap); + throw ce; + } catch (IOException e) { + DFSClient.LOG.warn("Exception while reading from " + + getCurrentBlock() + " of " + src + " from " + + currentNode, e); + throw e; + } + } + /** * Seek to a new arbitrary location */ @@ -416,7 +459,7 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, if (!curStripeRange.include(getOffsetInBlockGroup())) { readOneStripe(corruptedBlockMap); } - int ret = copy(strategy, off + result, realLen - result); + int ret = copyToTargetBuf(strategy, off + result, realLen - result); result += ret; pos += ret; } @@ -434,26 +477,6 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, return -1; } - private int readBuffer(BlockReader blockReader, - DatanodeInfo currentNode, ByteBufferStrategy readerStrategy, - Map> corruptedBlockMap) { - try { - return readerStrategy.doRead(blockReader, 0, 0); - } catch ( ChecksumException ce ) { - DFSClient.LOG.warn("Found Checksum error for " - + getCurrentBlock() + " from " + currentNode - + " at " + ce.getPos()); - // we want to remember which block replicas we have tried - addIntoCorruptedBlockMap(getCurrentBlock(), currentNode, - corruptedBlockMap); - } catch (IOException e) { - DFSClient.LOG.warn("Exception while reading from " - + getCurrentBlock() + " of " + src + " from " - + currentNode, e); - } - return -1; - } - /** * Copy the data from {@link #curStripeBuf} into the given buffer * @param strategy the ReaderStrategy containing the given buffer @@ -462,7 +485,7 @@ private int readBuffer(BlockReader blockReader, * @param length target length * @return number of bytes copied */ - private int copy(ReaderStrategy strategy, int offset, int length) { + private int copyToTargetBuf(ReaderStrategy strategy, int offset, int length) { final long offsetInBlk = getOffsetInBlockGroup(); int bufOffset = getStripedBufOffset(offsetInBlk); curStripeBuf.position(bufOffset); @@ -519,120 +542,19 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, cellSize, blockGroup, start, end, buf, offset); + CompletionService readService = new ExecutorCompletionService<>( + dfsClient.getStripedReadsThreadPool()); + // TODO handle null elements in blks (e.g., NN does not know locations for + // all the internal blocks) + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( + blockGroup, cellSize, dataBlkNum, parityBlkNum); for (AlignedStripe stripe : stripes) { - fetchOneStripe(blockGroup, buf, stripe, corruptedBlockMap); - } - } - - private void fetchOneStripe(LocatedStripedBlock blockGroup, - byte[] buf, AlignedStripe alignedStripe, Map> corruptedBlockMap) throws IOException { - Map, Integer> futures = new HashMap<>(); - CompletionService service = - new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); - if (alignedStripe.getSpanInBlock() == 0) { - DFSClient.LOG.warn("Trying to read an empty stripe from" + blockGroup); - return; - } - // Parse group to get chosen DN location - LocatedBlock[] blks = StripedBlockUtil. - parseStripedBlockGroup(blockGroup, cellSize, dataBlkNum, parityBlkNum); - for (short i = 0; i < dataBlkNum; i++) { - if (alignedStripe.chunks[i] != null - && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { - fetchOneStripingChunk(futures, service, blks[i], alignedStripe, i, - corruptedBlockMap); - } - } - // Input buffers for potential decode operation, which remains null until - // first read failure - byte[][] decodeInputs = null; - while (!futures.isEmpty()) { - try { - StripingChunkReadResult r = getNextCompletedStripedRead( - service, futures, 0); - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("Read task returned: " + r + ", for stripe " + alignedStripe); - } - StripingChunk returnedChunk = alignedStripe.chunks[r.index]; - Preconditions.checkNotNull(returnedChunk); - Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING); - if (r.state == StripingChunkReadResult.SUCCESSFUL) { - returnedChunk.state = StripingChunk.FETCHED; - alignedStripe.fetchedChunksNum++; - if (alignedStripe.fetchedChunksNum == dataBlkNum) { - clearFutures(futures.keySet()); - break; - } - } else { - returnedChunk.state = StripingChunk.MISSING; - alignedStripe.missingChunksNum++; - if (alignedStripe.missingChunksNum > parityBlkNum) { - clearFutures(futures.keySet()); - throw new IOException("Too many blocks are missing: " + alignedStripe); - } - // When seeing first missing block, initialize decode input buffers - if (decodeInputs == null) { - decodeInputs = initDecodeInputs(alignedStripe, dataBlkNum, parityBlkNum); - } - for (int i = 0; i < alignedStripe.chunks.length; i++) { - StripingChunk chunk = alignedStripe.chunks[i]; - Preconditions.checkNotNull(chunk); - if (chunk.state == StripingChunk.REQUESTED && i <= dataBlkNum) { - fetchOneStripingChunk(futures, service, blks[i], alignedStripe, i, - corruptedBlockMap); - } - } - } - } catch (InterruptedException ie) { - String err = "Read request interrupted"; - DFSClient.LOG.error(err); - clearFutures(futures.keySet()); - // Don't decode if read interrupted - throw new InterruptedIOException(err); - } - } - - if (alignedStripe.missingChunksNum > 0) { - finalizeDecodeInputs(decodeInputs, dataBlkNum, parityBlkNum, - alignedStripe); - decodeAndFillBuffer(decodeInputs, buf, alignedStripe, dataBlkNum, - parityBlkNum, decoder); + // Parse group to get chosen DN location + StripeReader preader = new PositionStripeReader(readService, stripe); + preader.readStripe(blks, corruptedBlockMap); } } - /** - * Schedule a single read request to an internal block - * @param block The internal block - * @param index Index of the internal block in the group - * @param corruptedBlockMap Map of corrupted blocks - */ - private void fetchOneStripingChunk(Map, Integer> futures, - final CompletionService service, final LocatedBlock block, - final AlignedStripe alignedStripe, final int index, - Map> corruptedBlockMap) { - DatanodeInfo loc = block.getLocations()[0]; - StorageType type = block.getStorageTypes()[0]; - DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( - loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), - type); - StripingChunk chunk = alignedStripe.chunks[index]; - chunk.state = StripingChunk.PENDING; - Callable readCallable = getFromOneDataNode(dnAddr, - block, alignedStripe.getOffsetInBlock(), - alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, - chunk.buf, chunk.getOffsets(), chunk.getLengths(), - corruptedBlockMap, index); - Future getFromDNRequest = service.submit(readCallable); - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("Submitting striped read request for " + index + - ". Info of the block: " + block + ", offset in block is " + - alignedStripe.getOffsetInBlock() + ", end is " + - (alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1)); - } - futures.put(getFromDNRequest, index); - } - private Callable getFromOneDataNode(final DNAddrPair datanode, final LocatedBlock block, final long start, final long end, final byte[] buf, final int[] offsets, final int[] lengths, @@ -655,21 +577,302 @@ public Void call() throws Exception { }; } - private void waitNextCompletion(CompletionService service, - Map, Integer> futures) throws InterruptedException { - if (futures.isEmpty()) { - throw new InterruptedException("Futures already empty"); + private abstract class StripeReader { + final Map, Integer> futures = new HashMap<>(); + final AlignedStripe alignedStripe; + final CompletionService service; + + StripeReader(CompletionService service, AlignedStripe alignedStripe) { + this.service = service; + this.alignedStripe = alignedStripe; + } + + /** submit reading chunk task */ + abstract void readChunk(final CompletionService service, + final LocatedBlock block, int chunkIndex, + Map> corruptedBlockMap); + + /** + * When seeing first missing block, initialize decode input buffers. + * Also prepare the reading for data blocks outside of the reading range. + */ + abstract void prepareDecodeInputs() throws IOException; + + /** + * Prepare reading for one more parity chunk. + */ + abstract void prepareParityChunk() throws IOException; + + abstract void decode(); + + abstract void updateState4SuccessRead(StripingChunkReadResult result); + + /** read the whole stripe. do decoding if necessary */ + void readStripe(LocatedBlock[] blocks, + Map> corruptedBlockMap) + throws IOException { + assert alignedStripe.getSpanInBlock() > 0; + for (short i = 0; i < dataBlkNum; i++) { + if (alignedStripe.chunks[i] != null + && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { + readChunk(service, blocks[i], i, corruptedBlockMap); + } + } + + // Input buffers for potential decode operation, which remains null until + // first read failure + while (!futures.isEmpty()) { + try { + StripingChunkReadResult r = getNextCompletedStripedRead(service, + futures, 0); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Read task returned: " + r + ", for stripe " + + alignedStripe); + } + StripingChunk returnedChunk = alignedStripe.chunks[r.index]; + Preconditions.checkNotNull(returnedChunk); + Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING); + + if (r.state == StripingChunkReadResult.SUCCESSFUL) { + returnedChunk.state = StripingChunk.FETCHED; + alignedStripe.fetchedChunksNum++; + updateState4SuccessRead(r); + if (alignedStripe.fetchedChunksNum == dataBlkNum) { + clearFutures(futures.keySet()); + break; + } + } else { + returnedChunk.state = StripingChunk.MISSING; + alignedStripe.missingChunksNum++; + if (alignedStripe.missingChunksNum > parityBlkNum) { + clearFutures(futures.keySet()); + throw new IOException("Too many blocks are missing: " + + alignedStripe); + } + + prepareDecodeInputs(); + prepareParityChunk(); + // close the corresponding reader + closeReader(r.index); + + for (int i = 0; i < alignedStripe.chunks.length; i++) { + StripingChunk chunk = alignedStripe.chunks[i]; + if (chunk != null && chunk.state == StripingChunk.REQUESTED) { + readChunk(service, blocks[i], i, corruptedBlockMap); + } + } + } + } catch (InterruptedException ie) { + String err = "Read request interrupted"; + DFSClient.LOG.error(err); + clearFutures(futures.keySet()); + // Don't decode if read interrupted + throw new InterruptedIOException(err); + } + } + + if (alignedStripe.missingChunksNum > 0) { + decode(); + } + } + } + + class PositionStripeReader extends StripeReader { + private byte[][] decodeInputs = null; + + PositionStripeReader(CompletionService service, + AlignedStripe alignedStripe) { + super(service, alignedStripe); + } + + @Override + void readChunk(final CompletionService service, + final LocatedBlock block, int chunkIndex, + Map> corruptedBlockMap) { + DatanodeInfo loc = block.getLocations()[0]; + StorageType type = block.getStorageTypes()[0]; + DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( + loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), + type); + StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + chunk.state = StripingChunk.PENDING; + Callable readCallable = getFromOneDataNode(dnAddr, + block, alignedStripe.getOffsetInBlock(), + alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, + chunk.byteArray.buf(), chunk.byteArray.getOffsets(), + chunk.byteArray.getLengths(), corruptedBlockMap, chunkIndex); + Future getFromDNRequest = service.submit(readCallable); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Submitting striped read request for " + chunkIndex + + ". Info of the block: " + block + ", offset in block is " + + alignedStripe.getOffsetInBlock() + ", end is " + + (alignedStripe.getOffsetInBlock() + + alignedStripe.getSpanInBlock() - 1)); + } + futures.put(getFromDNRequest, chunkIndex); + } + + @Override + void updateState4SuccessRead(StripingChunkReadResult r) {} + + @Override + void prepareDecodeInputs() { + if (decodeInputs == null) { + decodeInputs = initDecodeInputs(alignedStripe, dataBlkNum, parityBlkNum); + } + } + + @Override + void prepareParityChunk() { + for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { + if (alignedStripe.chunks[i] == null) { + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); + alignedStripe.chunks[i].addByteArraySlice(0, + (int) alignedStripe.getSpanInBlock()); + break; + } + } + } + + @Override + void decode() { + finalizeDecodeInputs(decodeInputs, dataBlkNum, parityBlkNum, + alignedStripe); + decodeAndFillBuffer(decodeInputs, alignedStripe, dataBlkNum, + parityBlkNum, decoder); + } + } + + class StatefulStripeReader extends StripeReader { + ByteBuffer[] decodeInputs; + final LocatedBlock[] targetBlocks; + + StatefulStripeReader(CompletionService service, + AlignedStripe alignedStripe, LocatedBlock[] targetBlocks) { + super(service, alignedStripe); + this.targetBlocks = targetBlocks; + } + + @Override + void readChunk(final CompletionService service, + final LocatedBlock block, int chunkIndex, Map> corruptedBlockMap) { + StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + chunk.state = StripingChunk.PENDING; + ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer); + Callable readCallable = readCell(blockReaders[chunkIndex], + currentNodes[chunkIndex], blockReaderOffsets[chunkIndex], + alignedStripe.getOffsetInBlock(), strategy, + chunk.byteBuffer.remaining(), corruptedBlockMap); + Future request = readingService.submit(readCallable); + futures.put(request, chunkIndex); + } + + @Override + void updateState4SuccessRead(StripingChunkReadResult result) { + Preconditions.checkArgument( + result.state == StripingChunkReadResult.SUCCESSFUL); + blockReaderOffsets[result.index] = + alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock(); + } + + @Override + void prepareDecodeInputs() throws IOException { + if (decodeInputs == null) { + decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum]; + ByteBuffer cur = curStripeBuf.duplicate(); + StripedBlockUtil.VerticalRange range = alignedStripe.range; + for (int i = 0; i < dataBlkNum; i++) { + cur.limit(cur.capacity()); + int pos = (int) (range.offsetInBlock % cellSize + cellSize * i); + cur.position(pos); + cur.limit((int) (pos + range.spanInBlock)); + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, + parityBlkNum); + decodeInputs[decodeIndex] = cur.slice(); + if (alignedStripe.chunks[i] == null) { + alignedStripe.chunks[i] = + new StripingChunk(decodeInputs[decodeIndex]); + } + } + } + } + + @Override + void prepareParityChunk() throws IOException { + for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { + if (alignedStripe.chunks[i] == null) { + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, + parityBlkNum); + decodeInputs[decodeIndex] = ByteBuffer.allocateDirect( + (int) alignedStripe.range.spanInBlock); + alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); + if (blockReaders[i] == null) { + prepareParityBlockReader(i); + } + break; + } + } + } + + private void prepareParityBlockReader(int i) throws IOException { + // prepare the block reader for the parity chunk + LocatedBlock targetBlock = targetBlocks[i]; + if (targetBlock != null) { + final long offsetInBlock = alignedStripe.getOffsetInBlock(); + DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); + if (retval != null) { + currentNodes[i] = retval.info; + blockReaders[i] = getBlockReaderWithRetry(targetBlock, + offsetInBlock, targetBlock.getBlockSize() - offsetInBlock, + retval.addr, retval.storageType, retval.info, + DFSStripedInputStream.this.getPos(), retry); + blockReaderOffsets[i] = offsetInBlock; + } + } + } + + @Override + void decode() { + // TODO no copy for data chunks. this depends on HADOOP-12047 for some + // decoders to work + final int span = (int) alignedStripe.getSpanInBlock(); + for (int i = 0; i < alignedStripe.chunks.length; i++) { + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.ALLZERO) { + for (int j = 0; j < span; j++) { + decodeInputs[decodeIndex].put((byte) 0); + } + decodeInputs[decodeIndex].flip(); + } else if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.FETCHED) { + decodeInputs[decodeIndex].position(0); + decodeInputs[decodeIndex].limit(span); + } + } + int[] decodeIndices = new int[parityBlkNum]; + int pos = 0; + for (int i = 0; i < alignedStripe.chunks.length; i++) { + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.MISSING) { + decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + } + } + decodeIndices = Arrays.copyOf(decodeIndices, pos); + + final int decodeChunkNum = decodeIndices.length; + ByteBuffer[] outputs = new ByteBuffer[decodeChunkNum]; + for (int i = 0; i < decodeChunkNum; i++) { + outputs[i] = decodeInputs[decodeIndices[i]]; + outputs[i].position(0); + outputs[i].limit((int) alignedStripe.range.spanInBlock); + decodeInputs[decodeIndices[i]] = null; + } + + decoder.decode(decodeInputs, decodeIndices, outputs); } - Future future = null; - try { - future = service.take(); - future.get(); - futures.remove(future); - } catch (ExecutionException | CancellationException e) { - // already logged in the Callable - futures.remove(future); - } - throw new InterruptedException("let's retry"); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 1db2045741f03..a29e8e35185ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -33,6 +33,7 @@ import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import java.nio.ByteBuffer; import java.util.*; import java.io.IOException; import java.util.concurrent.CancellationException; @@ -79,7 +80,6 @@ public class StripedBlockUtil { public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, int cellSize, int dataBlkNum, int parityBlkNum) { int locatedBGSize = bg.getBlockIndices().length; - // TODO not considering missing blocks for now, only identify data blocks LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum]; for (short i = 0; i < locatedBGSize; i++) { final int idx = bg.getBlockIndices()[i]; @@ -212,7 +212,7 @@ public static StripingChunkReadResult getNextCompletedStripedRead( return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT); } } catch (ExecutionException e) { - DFSClient.LOG.error("ExecutionException " + e); + DFSClient.LOG.warn("ExecutionException " + e); return new StripingChunkReadResult(futures.remove(future), StripingChunkReadResult.FAILED); } catch (CancellationException e) { @@ -253,12 +253,13 @@ public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum) { byte[][] decodeInputs = new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()]; - for (int i = 0; i < alignedStripe.chunks.length; i++) { + // read the full data aligned stripe + for (int i = 0; i < dataBlkNum; i++) { if (alignedStripe.chunks[i] == null) { final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); - alignedStripe.chunks[i].offsetsInBuf.add(0); - alignedStripe.chunks[i].lengthsInBuf.add((int) alignedStripe.getSpanInBlock()); + alignedStripe.chunks[i].addByteArraySlice(0, + (int) alignedStripe.getSpanInBlock()); } } return decodeInputs; @@ -276,14 +277,9 @@ public static void finalizeDecodeInputs(final byte[][] decodeInputs, for (int i = 0; i < alignedStripe.chunks.length; i++) { final StripingChunk chunk = alignedStripe.chunks[i]; final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); - if (chunk.state == StripingChunk.FETCHED) { - int posInBuf = 0; - for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { - System.arraycopy(chunk.buf, chunk.offsetsInBuf.get(j), - decodeInputs[decodeIndex], posInBuf, chunk.lengthsInBuf.get(j)); - posInBuf += chunk.lengthsInBuf.get(j); - } - } else if (chunk.state == StripingChunk.ALLZERO) { + if (chunk != null && chunk.state == StripingChunk.FETCHED) { + chunk.copyTo(decodeInputs[decodeIndex]); + } else if (chunk != null && chunk.state == StripingChunk.ALLZERO) { Arrays.fill(decodeInputs[decodeIndex], (byte) 0); } else { decodeInputs[decodeIndex] = null; @@ -315,13 +311,14 @@ public static int convertDecodeIndexBack(int index, int dataBlkNum, * Decode based on the given input buffers and schema. */ public static void decodeAndFillBuffer(final byte[][] decodeInputs, - byte[] buf, AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum, + AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum, RawErasureDecoder decoder) { // Step 1: prepare indices and output buffers for missing data units int[] decodeIndices = new int[parityBlkNum]; int pos = 0; for (int i = 0; i < alignedStripe.chunks.length; i++) { - if (alignedStripe.chunks[i].state == StripingChunk.MISSING){ + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.MISSING){ decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum); } } @@ -338,14 +335,56 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, dataBlkNum, parityBlkNum); StripingChunk chunk = alignedStripe.chunks[missingBlkIdx]; if (chunk.state == StripingChunk.MISSING) { - int srcPos = 0; - for (int j = 0; j < chunk.offsetsInBuf.size(); j++) { - System.arraycopy(decodeOutputs[i], srcPos, buf, - chunk.offsetsInBuf.get(j), chunk.lengthsInBuf.get(j)); - srcPos += chunk.lengthsInBuf.get(j); + chunk.copyFrom(decodeOutputs[i]); + } + } + } + + /** + * Similar functionality with {@link #divideByteRangeIntoStripes}, but is used + * by stateful read and uses ByteBuffer as reading target buffer. Besides the + * read range is within a single stripe thus the calculation logic is simpler. + */ + public static AlignedStripe[] divideOneStripe(ECSchema ecSchema, + int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, + long rangeEndInBlockGroup, ByteBuffer buf) { + final int dataBlkNum = ecSchema.getNumDataUnits(); + // Step 1: map the byte range to StripingCells + StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize, + blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup); + + // Step 2: get the unmerged ranges on each internal block + VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize, + cells); + + // Step 3: merge into stripes + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); + + // Step 4: calculate each chunk's position in destination buffer. Since the + // whole read range is within a single stripe, the logic is simpler here. + int bufOffset = (int) (rangeStartInBlockGroup % (cellSize * dataBlkNum)); + for (StripingCell cell : cells) { + long cellStart = cell.idxInInternalBlk * cellSize + cell.offset; + long cellEnd = cellStart + cell.size - 1; + for (AlignedStripe s : stripes) { + long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1; + long overlapStart = Math.max(cellStart, s.getOffsetInBlock()); + long overlapEnd = Math.min(cellEnd, stripeEnd); + int overLapLen = (int) (overlapEnd - overlapStart + 1); + if (overLapLen > 0) { + Preconditions.checkState(s.chunks[cell.idxInStripe] == null); + final int pos = (int) (bufOffset + overlapStart - cellStart); + buf.position(pos); + buf.limit(pos + overLapLen); + s.chunks[cell.idxInStripe] = new StripingChunk(buf.slice()); } } + bufOffset += cell.size; } + + // Step 5: prepare ALLZERO blocks + prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum); + return stripes; } /** @@ -369,7 +408,7 @@ public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, int offsetInBuf) { // Step 0: analyze range and calculate basic parameters - int dataBlkNum = ecSchema.getNumDataUnits(); + final int dataBlkNum = ecSchema.getNumDataUnits(); // Step 1: map the byte range to StripingCells StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize, @@ -386,7 +425,7 @@ public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf); // Step 5: prepare ALLZERO blocks - prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum); + prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum); return stripes; } @@ -403,23 +442,25 @@ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, Preconditions.checkArgument( rangeStartInBlockGroup <= rangeEndInBlockGroup && rangeEndInBlockGroup < blockGroup.getBlockSize()); - int len = (int) (rangeEndInBlockGroup - rangeStartInBlockGroup + 1); + long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1; int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize); int numCells = lastCellIdxInBG - firstCellIdxInBG + 1; StripingCell[] cells = new StripingCell[numCells]; - cells[0] = new StripingCell(ecSchema, cellSize, firstCellIdxInBG); - cells[numCells - 1] = new StripingCell(ecSchema, cellSize, lastCellIdxInBG); - cells[0].offset = (int) (rangeStartInBlockGroup % cellSize); - cells[0].size = - Math.min(cellSize - (int) (rangeStartInBlockGroup % cellSize), len); + final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize); + final int firstCellSize = + (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len); + cells[0] = new StripingCell(ecSchema, firstCellSize, firstCellIdxInBG, + firstCellOffset); if (lastCellIdxInBG != firstCellIdxInBG) { - cells[numCells - 1].size = (int) (rangeEndInBlockGroup % cellSize) + 1; + final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1; + cells[numCells - 1] = new StripingCell(ecSchema, lastCellSize, + lastCellIdxInBG, 0); } for (int i = 1; i < numCells - 1; i++) { - cells[i] = new StripingCell(ecSchema, cellSize, i + firstCellIdxInBG); + cells[i] = new StripingCell(ecSchema, cellSize, i + firstCellIdxInBG, 0); } return cells; @@ -438,8 +479,8 @@ public static long[] getStartOffsetsForInternalBlocks(ECSchema ecSchema, long[] startOffsets = new long[dataBlkNum + parityBlkNum]; Arrays.fill(startOffsets, -1L); int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); - StripingCell firstCell = new StripingCell(ecSchema, cellSize, firstCellIdxInBG); - firstCell.offset = (int) (rangeStartInBlockGroup % cellSize); + StripingCell firstCell = new StripingCell(ecSchema, cellSize, + firstCellIdxInBG, (int) (rangeStartInBlockGroup % cellSize)); startOffsets[firstCell.idxInStripe] = firstCell.idxInInternalBlk * cellSize + firstCell.offset; long earliestStart = startOffsets[firstCell.idxInStripe]; @@ -448,7 +489,7 @@ public static long[] getStartOffsetsForInternalBlocks(ECSchema ecSchema, if (idx * (long) cellSize >= blockGroup.getBlockSize()) { break; } - StripingCell cell = new StripingCell(ecSchema, cellSize, idx); + StripingCell cell = new StripingCell(ecSchema, cellSize, idx, 0); startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * (long) cellSize; if (startOffsets[cell.idxInStripe] < earliestStart) { earliestStart = startOffsets[cell.idxInStripe]; @@ -563,10 +604,8 @@ private static void calcualteChunkPositionsInBuf(int cellSize, if (s.chunks[cell.idxInStripe] == null) { s.chunks[cell.idxInStripe] = new StripingChunk(buf); } - - s.chunks[cell.idxInStripe].offsetsInBuf. - add((int)(offsetInBuf + done + overlapStart - cellStart)); - s.chunks[cell.idxInStripe].lengthsInBuf.add(overLapLen); + s.chunks[cell.idxInStripe].addByteArraySlice( + (int)(offsetInBuf + done + overlapStart - cellStart), overLapLen); } done += cell.size; } @@ -577,15 +616,14 @@ private static void calcualteChunkPositionsInBuf(int cellSize, * size, the chunk should be treated as zero bytes in decoding. */ private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, - byte[] buf, AlignedStripe[] stripes, int cellSize, int dataBlkNum) { + AlignedStripe[] stripes, int cellSize, int dataBlkNum) { for (AlignedStripe s : stripes) { for (int i = 0; i < dataBlkNum; i++) { long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(), cellSize, dataBlkNum, i); if (internalBlkLen <= s.getOffsetInBlock()) { Preconditions.checkState(s.chunks[i] == null); - s.chunks[i] = new StripingChunk(buf); - s.chunks[i].state = StripingChunk.ALLZERO; + s.chunks[i] = new StripingChunk(); // chunk state is set to ALLZERO } } } @@ -615,7 +653,7 @@ private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, */ @VisibleForTesting static class StripingCell { - public final ECSchema schema; + final ECSchema schema; /** Logical order in a block group, used when doing I/O to a block group */ final int idxInBlkGroup; final int idxInInternalBlk; @@ -626,27 +664,17 @@ static class StripingCell { * {@link #size} variable represent the start offset and size of the * overlap. */ - int offset; - int size; + final int offset; + final int size; - StripingCell(ECSchema ecSchema, int cellSize, int idxInBlkGroup) { + StripingCell(ECSchema ecSchema, int cellSize, int idxInBlkGroup, + int offset) { this.schema = ecSchema; this.idxInBlkGroup = idxInBlkGroup; this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits(); this.idxInStripe = idxInBlkGroup - this.idxInInternalBlk * ecSchema.getNumDataUnits(); - this.offset = 0; - this.size = cellSize; - } - - StripingCell(ECSchema ecSchema, int cellSize, int idxInInternalBlk, - int idxInStripe) { - this.schema = ecSchema; - this.idxInInternalBlk = idxInInternalBlk; - this.idxInStripe = idxInStripe; - this.idxInBlkGroup = - idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe; - this.offset = 0; + this.offset = offset; this.size = cellSize; } } @@ -700,11 +728,6 @@ public AlignedStripe(long offsetInBlock, long length, int width) { this.chunks = new StripingChunk[width]; } - public AlignedStripe(VerticalRange range, int width) { - this.range = range; - this.chunks = new StripingChunk[width]; - } - public boolean include(long pos) { return range.include(pos); } @@ -777,10 +800,6 @@ public boolean include(long pos) { * |REQUESTED| |REQUESTED| ALLZERO | |null| |null| <- AlignedStripe2 * +---------+ +---------+ | +----+ +----+ * <----------- data blocks ------------> | <--- parity ---> - * - * The class also carries {@link #buf}, {@link #offsetsInBuf}, and - * {@link #lengthsInBuf} to define how read task for this chunk should - * deliver the returned data. */ public static class StripingChunk { /** Chunk has been successfully fetched */ @@ -808,11 +827,49 @@ public static class StripingChunk { * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ... */ public int state = REQUESTED; - public byte[] buf; - public List offsetsInBuf; - public List lengthsInBuf; + + public final ChunkByteArray byteArray; + public final ByteBuffer byteBuffer; public StripingChunk(byte[] buf) { + this.byteArray = new ChunkByteArray(buf); + byteBuffer = null; + } + + public StripingChunk(ByteBuffer buf) { + this.byteArray = null; + this.byteBuffer = buf; + } + + public StripingChunk() { + this.byteArray = null; + this.byteBuffer = null; + this.state = ALLZERO; + } + + public void addByteArraySlice(int offset, int length) { + assert byteArray != null; + byteArray.offsetsInBuf.add(offset); + byteArray.lengthsInBuf.add(length); + } + + void copyTo(byte[] target) { + assert byteArray != null; + byteArray.copyTo(target); + } + + void copyFrom(byte[] src) { + assert byteArray != null; + byteArray.copyFrom(src); + } + } + + public static class ChunkByteArray { + private final byte[] buf; + private final List offsetsInBuf; + private final List lengthsInBuf; + + ChunkByteArray(byte[] buf) { this.buf = buf; this.offsetsInBuf = new ArrayList<>(); this.lengthsInBuf = new ArrayList<>(); @@ -833,6 +890,28 @@ public int[] getLengths() { } return lens; } + + public byte[] buf() { + return buf; + } + + void copyTo(byte[] target) { + int posInBuf = 0; + for (int i = 0; i < offsetsInBuf.size(); i++) { + System.arraycopy(buf, offsetsInBuf.get(i), + target, posInBuf, lengthsInBuf.get(i)); + posInBuf += lengthsInBuf.get(i); + } + } + + void copyFrom(byte[] src) { + int srcPos = 0; + for (int j = 0; j < offsetsInBuf.size(); j++) { + System.arraycopy(src, srcPos, buf, offsetsInBuf.get(j), + lengthsInBuf.get(j)); + srcPos += lengthsInBuf.get(j); + } + } } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index 54367d743af63..23697040db733 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -32,7 +32,7 @@ public class StripedFileTestUtil { static final int blockSize = cellSize * stripesPerBlock; static final int numDNs = dataBlocks + parityBlocks + 2; - static final Random r = new Random(); + static final Random random = new Random(); static byte[] generateBytes(int cnt) { byte[] bytes = new byte[cnt]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index a28f88ef8bbd4..0201d071ab275 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -29,6 +29,7 @@ import org.junit.Test; import java.io.IOException; +import java.nio.ByteBuffer; import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; @@ -58,28 +59,28 @@ public void tearDown() throws IOException { } @Test - public void testWritePreadWithDNFailure1() throws IOException { - testWritePreadWithDNFailure("/foo", cellSize * (dataBlocks + 2), 0); + public void testReadWithDNFailure1() throws IOException { + testReadWithDNFailure("/foo", cellSize * (dataBlocks + 2), 0); } @Test - public void testWritePreadWithDNFailure2() throws IOException { - testWritePreadWithDNFailure("/foo", cellSize * (dataBlocks + 2), cellSize * 5); + public void testReadWithDNFailure2() throws IOException { + testReadWithDNFailure("/foo", cellSize * (dataBlocks + 2), cellSize * 5); } @Test - public void testWritePreadWithDNFailure3() throws IOException { - testWritePreadWithDNFailure("/foo", cellSize * dataBlocks, 0); + public void testReadWithDNFailure3() throws IOException { + testReadWithDNFailure("/foo", cellSize * dataBlocks, 0); } - private void testWritePreadWithDNFailure(String file, int fileSize, int startOffsetInFile) - throws IOException { + private void testReadWithDNFailure(String file, int fileSize, + int startOffsetInFile) throws IOException { final int failedDNIdx = 2; Path testPath = new Path(file); final byte[] bytes = StripedFileTestUtil.generateBytes(fileSize); DFSTestUtil.writeFile(fs, testPath, bytes); - // shut down the DN that holds the last internal data block + // shut down the DN that holds an internal data block BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, cellSize); String name = (locs[0].getNames())[failedDNIdx]; @@ -99,14 +100,30 @@ private void testWritePreadWithDNFailure(String file, int fileSize, int startOff fileSize - startOffsetInFile, readLen); byte[] expected = new byte[readLen]; - for (int i = startOffsetInFile; i < fileSize; i++) { - expected[i - startOffsetInFile] = StripedFileTestUtil.getByte(i); - } + System.arraycopy(bytes, startOffsetInFile, expected, 0, + fileSize - startOffsetInFile); for (int i = startOffsetInFile; i < fileSize; i++) { Assert.assertEquals("Byte at " + i + " should be the same", expected[i - startOffsetInFile], buf[i - startOffsetInFile]); } } + + // stateful read + ByteBuffer result = ByteBuffer.allocate(fileSize); + ByteBuffer buf = ByteBuffer.allocate(1024); + int readLen = 0; + int ret; + try (FSDataInputStream in = fs.open(testPath)) { + while ((ret = in.read(buf)) >= 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } + Assert.assertEquals("The length of file should be the same to write size", + fileSize, readLen); + Assert.assertArrayEquals(bytes, result.array()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index e2e52467a625c..272650d358c35 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -17,17 +17,21 @@ */ package org.apache.hadoop.hdfs; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.web.ByteRangeInputStream; import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; -import org.junit.AfterClass; +import org.junit.After; import org.junit.Assert; -import org.junit.BeforeClass; +import org.junit.Before; import org.junit.Test; import java.io.EOFException; @@ -41,12 +45,13 @@ import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock; public class TestWriteReadStripedFile { + public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class); private static MiniDFSCluster cluster; private static FileSystem fs; private static Configuration conf; - @BeforeClass - public static void setup() throws IOException { + @Before + public void setup() throws IOException { conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); @@ -55,8 +60,8 @@ public static void setup() throws IOException { fs = cluster.getFileSystem(); } - @AfterClass - public static void tearDown() throws IOException { + @After + public void tearDown() throws IOException { if (cluster != null) { cluster.shutdown(); } @@ -65,75 +70,98 @@ public static void tearDown() throws IOException { @Test public void testFileEmpty() throws IOException { testOneFileUsingDFSStripedInputStream("/EmptyFile", 0); + testOneFileUsingDFSStripedInputStream("/EmptyFile2", 0, true); } @Test public void testFileSmallerThanOneCell1() throws IOException { testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", 1, true); } @Test public void testFileSmallerThanOneCell2() throws IOException { testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", cellSize - 1, + true); } @Test public void testFileEqualsWithOneCell() throws IOException { testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize); + testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell2", cellSize, true); } @Test public void testFileSmallerThanOneStripe1() throws IOException { testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize * dataBlocks - 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2", + cellSize * dataBlocks - 1, true); } @Test public void testFileSmallerThanOneStripe2() throws IOException { testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize + 123); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2", + cellSize + 123, true); } @Test public void testFileEqualsWithOneStripe() throws IOException { testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", cellSize * dataBlocks); + testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe2", + cellSize * dataBlocks, true); } @Test public void testFileMoreThanOneStripe1() throws IOException { testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", cellSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe12", + cellSize * dataBlocks + 123, true); } @Test public void testFileMoreThanOneStripe2() throws IOException { testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", cellSize * dataBlocks + cellSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe22", + cellSize * dataBlocks + cellSize * dataBlocks + 123, true); } @Test public void testLessThanFullBlockGroup() throws IOException { testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup", cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); + testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup2", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize, true); } @Test public void testFileFullBlockGroup() throws IOException { testOneFileUsingDFSStripedInputStream("/FullBlockGroup", blockSize * dataBlocks); + testOneFileUsingDFSStripedInputStream("/FullBlockGroup2", + blockSize * dataBlocks, true); } @Test public void testFileMoreThanABlockGroup1() throws IOException { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup12", + blockSize * dataBlocks + 123, true); } @Test public void testFileMoreThanABlockGroup2() throws IOException { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup22", + blockSize * dataBlocks + cellSize + 123, true); } @@ -142,6 +170,9 @@ public void testFileMoreThanABlockGroup3() throws IOException { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3", blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup32", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123, true); } private void assertSeekAndRead(FSDataInputStream fsdis, int pos, @@ -158,12 +189,23 @@ private void assertSeekAndRead(FSDataInputStream fsdis, int pos, private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) throws IOException { + testOneFileUsingDFSStripedInputStream(src, fileLength, false); + } + + private void testOneFileUsingDFSStripedInputStream(String src, int fileLength, + boolean withDataNodeFailure) throws IOException { final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); Path srcPath = new Path(src); DFSTestUtil.writeFile(fs, srcPath, new String(expected)); verifyLength(fs, srcPath, fileLength); + if (withDataNodeFailure) { + int dnIndex = 1; // TODO: StripedFileTestUtil.random.nextInt(dataBlocks); + LOG.info("stop DataNode " + dnIndex); + stopDataNode(srcPath, dnIndex); + } + byte[] smallBuf = new byte[1024]; byte[] largeBuf = new byte[fileLength + 100]; verifyPread(fs, srcPath, fileLength, expected, largeBuf); @@ -177,6 +219,21 @@ private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) ByteBuffer.allocate(1024)); } + private void stopDataNode(Path path, int failedDNIdx) + throws IOException { + BlockLocation[] locs = fs.getFileBlockLocations(path, 0, cellSize); + if (locs != null && locs.length > 0) { + String name = (locs[0].getNames())[failedDNIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + break; + } + } + } + } + @Test public void testWriteReadUsingWebHdfs() throws Exception { int fileLength = blockSize * dataBlocks + cellSize + 123; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java index 5a1c3fc096598..5d85073994f3b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -152,7 +152,7 @@ private byte[][] createInternalBlkBuffers(int bgSize) { int done = 0; while (done < bgSize) { Preconditions.checkState(done % CELLSIZE == 0); - StripingCell cell = new StripingCell(SCEHMA, CELLSIZE, done / CELLSIZE); + StripingCell cell = new StripingCell(SCEHMA, CELLSIZE, done / CELLSIZE, 0); int idxInStripe = cell.idxInStripe; int size = Math.min(CELLSIZE, bgSize - done); for (int i = 0; i < size; i++) { @@ -176,8 +176,7 @@ public void testParseDummyStripedBlock() { assertFalse(blocks[i].isStriped()); assertEquals(i, BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock())); - assertEquals(i * CELLSIZE, blocks[i].getStartOffset()); - /** TODO: properly define {@link LocatedBlock#offset} for internal blocks */ + assertEquals(0, blocks[i].getStartOffset()); assertEquals(1, blocks[i].getLocations().length); assertEquals(i, blocks[i].getLocations()[0].getIpcPort()); assertEquals(i, blocks[i].getLocations()[0].getXferPort()); @@ -256,11 +255,12 @@ public void testDivideByteRangeIntoStripes() { continue; } int done = 0; - for (int j = 0; j < chunk.getLengths().length; j++) { + for (int j = 0; j < chunk.byteArray.getLengths().length; j++) { System.arraycopy(internalBlkBufs[i], (int) stripe.getOffsetInBlock() + done, assembled, - chunk.getOffsets()[j], chunk.getLengths()[j]); - done += chunk.getLengths()[j]; + chunk.byteArray.getOffsets()[j], + chunk.byteArray.getLengths()[j]); + done += chunk.byteArray.getLengths()[j]; } } } From 2eee19cd1d8e16f0631d78474e387ca4c526f47e Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 5 Jun 2015 13:39:37 -0700 Subject: [PATCH 151/212] HDFS-8460. Erasure Coding: stateful read result doesn't match data occasionally because of flawed test. Contributed by Walter Su. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java | 2 ++ .../org/apache/hadoop/hdfs/TestDFSStripedInputStream.java | 5 +++++ 3 files changed, 10 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9b05d2bc7013b..61d7a763af534 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -286,3 +286,6 @@ HDFS-8319. Erasure Coding: support decoding for stateful read. (Jing Zhao via zhz) + + HDFS-8460. Erasure Coding: stateful read result doesn't match data + occasionally because of flawed test. (Walter Su via zhz) 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 db230e3266415..b9ded80ac9204 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 @@ -1859,6 +1859,7 @@ public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock( /** * Creates the metadata of a file in striped layout. This method only * manipulates the NameNode state without injecting data to DataNode. + * You should disable periodical heartbeat before use this. * @param file Path of the file to create * @param dir Parent path of the file * @param numBlocks Number of striped block groups to add to the file @@ -1908,6 +1909,7 @@ public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir * Adds a striped block group to a file. This method only manipulates NameNode * states of the file and the block without injecting data to DataNode. * It does mimic block reports. + * You should disable periodical heartbeat before use this. * @param dataNodes List DataNodes to host the striped block group * @param previous Previous block in the file * @param numStripes Number of stripes in each block group diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index de43441d671a6..b53983bc5a85e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -32,6 +32,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; @@ -71,6 +73,9 @@ public void setup() throws IOException { cluster = new MiniDFSCluster.Builder(conf).numDataNodes( DATA_BLK_NUM + PARITY_BLK_NUM).build(); cluster.waitActive(); + for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); + } fs = cluster.getFileSystem(); fs.mkdirs(dirPath); fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE); From c41b02cc0058239657263d45d969e71b14e6a589 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Mon, 8 Jun 2015 20:24:17 +0800 Subject: [PATCH 152/212] HADOOP-12065 Using more meaningful keys in EC schema. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 4 +++- .../java/org/apache/hadoop/io/erasurecode/ECSchema.java | 4 ++-- .../apache/hadoop/io/erasurecode/TestSchemaLoader.java | 8 ++++---- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 3559436a0760f..505eabdbcd194 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -64,4 +64,6 @@ (Kai Zheng) HADOOP-12011. Allow to dump verbose information to ease debugging in raw erasure coders - (Kai Zheng) \ No newline at end of file + (Kai Zheng) + + HADOOP-12065. Using more meaningful keys in EC schema. (Kai Zheng) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index fdc569ee9a9ff..1e07d3d04f761 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -25,8 +25,8 @@ * Erasure coding schema to housekeeper relevant information. */ public final class ECSchema { - public static final String NUM_DATA_UNITS_KEY = "k"; - public static final String NUM_PARITY_UNITS_KEY = "m"; + public static final String NUM_DATA_UNITS_KEY = "numDataUnits"; + public static final String NUM_PARITY_UNITS_KEY = "numParityUnits"; public static final String CODEC_NAME_KEY = "codec"; /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java index 939fa9b861304..50d20918894e8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java @@ -40,13 +40,13 @@ public void testLoadSchema() throws Exception { out.println(""); out.println(""); out.println(" "); - out.println(" 6"); - out.println(" 3"); + out.println(" 6"); + out.println(" 3"); out.println(" RS"); out.println(" "); out.println(" "); - out.println(" 10"); - out.println(" 4"); + out.println(" 10"); + out.println(" 4"); out.println(" RS"); out.println(" "); out.println(""); From e299fe86b889968a0093f9f9b097dd71b4f49e88 Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Wed, 10 Jun 2015 15:35:26 +0800 Subject: [PATCH 153/212] HDFS-8557 Allow to configure RS and XOR raw coders. Contributed by Kai Zheng --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 4 +- .../hadoop/fs/CommonConfigurationKeys.java | 6 +- .../hadoop/io/erasurecode/CodecUtil.java | 144 ++++++++++++++++++ .../coder/AbstractErasureCoder.java | 67 +------- .../erasurecode/coder/RSErasureDecoder.java | 11 +- .../erasurecode/coder/RSErasureEncoder.java | 9 +- .../erasurecode/coder/XORErasureDecoder.java | 10 +- .../erasurecode/coder/XORErasureEncoder.java | 10 +- .../hadoop/hdfs/DFSStripedInputStream.java | 23 ++- .../hadoop/hdfs/DFSStripedOutputStream.java | 16 +- .../erasurecode/ErasureCodingWorker.java | 3 +- .../hdfs/TestDFSStripedInputStream.java | 7 +- .../hdfs/TestDFSStripedOutputStream.java | 22 ++- ...TestDFSStripedOutputStreamWithFailure.java | 2 +- 14 files changed, 216 insertions(+), 118 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 505eabdbcd194..9ccd3a7d3418f 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -66,4 +66,6 @@ HADOOP-12011. Allow to dump verbose information to ease debugging in raw erasure coders (Kai Zheng) - HADOOP-12065. Using more meaningful keys in EC schema. (Kai Zheng) \ No newline at end of file + HADOOP-12065. Using more meaningful keys in EC schema. (Kai Zheng) + + HDFS-8557. Allow to configure RS and XOR raw coders (Kai Zheng) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 3f2871b023bd9..9588254ed5943 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -143,10 +143,14 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** Supported erasure codec classes */ public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs"; - /** Raw coder factory for the RS codec */ + /** Raw coder factory for the RS codec. */ public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY = "io.erasurecode.codec.rs.rawcoder"; + /** Raw coder factory for the XOR codec. */ + public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY = + "io.erasurecode.codec.xor.rawcoder"; + /** * Service Authorization */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java new file mode 100644 index 0000000000000..5d226248ef214 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java @@ -0,0 +1,144 @@ +/** + * 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.io.erasurecode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.erasurecode.rawcoder.*; + +/** + * A codec utility. + */ +public final class CodecUtil { + + private CodecUtil() {} + + /** + * Create RS raw encoder according to configuration. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @return raw encoder + */ + public static RawErasureEncoder createRSRawEncoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + true, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new RSRawEncoder(numDataUnits, numParityUnits); + } + + return (RawErasureEncoder) rawCoder; + } + + /** + * Create RS raw decoder according to configuration. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @return raw decoder + */ + public static RawErasureDecoder createRSRawDecoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + false, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new RSRawDecoder(numDataUnits, numParityUnits); + } + + return (RawErasureDecoder) rawCoder; + } + + /** + * Create XOR raw encoder according to configuration. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @return raw encoder + */ + public static RawErasureEncoder createXORRawEncoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY, + true, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new XORRawEncoder(numDataUnits, numParityUnits); + } + + return (RawErasureEncoder) rawCoder; + } + + /** + * Create XOR raw decoder according to configuration. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @return raw decoder + */ + public static RawErasureDecoder createXORRawDecoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY, + false, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new XORRawDecoder(numDataUnits, numParityUnits); + } + + return (RawErasureDecoder) rawCoder; + } + + /** + * Create raw coder using specified conf and raw coder factory key. + * @param conf + * @param rawCoderFactoryKey + * @param isEncoder + * @param numDataUnits + * @param numParityUnits + * @return raw coder + */ + public static RawErasureCoder createRawCoder(Configuration conf, + String rawCoderFactoryKey, boolean isEncoder, int numDataUnits, + int numParityUnits) { + + if (conf == null) { + return null; + } + + Class factClass = null; + factClass = conf.getClass(rawCoderFactoryKey, + factClass, RawErasureCoderFactory.class); + + if (factClass == null) { + return null; + } + + RawErasureCoderFactory fact; + try { + fact = factClass.newInstance(); + } catch (InstantiationException e) { + throw new RuntimeException("Failed to create raw coder", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Failed to create raw coder", e); + } + + return isEncoder ? fact.createEncoder(numDataUnits, numParityUnits) : + fact.createDecoder(numDataUnits, numParityUnits); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java index c572badef2e12..5cd0ee8c18300 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java @@ -17,13 +17,8 @@ */ package org.apache.hadoop.io.erasurecode.coder; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; /** * A common class of basic facilities to be shared by encoder and decoder @@ -36,73 +31,13 @@ public abstract class AbstractErasureCoder private final int numDataUnits; private final int numParityUnits; - /** - * Create raw decoder using the factory specified by rawCoderFactoryKey - * @param rawCoderFactoryKey - * @return raw decoder - */ - protected RawErasureDecoder createRawDecoder( - String rawCoderFactoryKey, int dataUnitsCount, int parityUnitsCount) { - RawErasureCoder rawCoder = createRawCoder(getConf(), - rawCoderFactoryKey, false, dataUnitsCount, parityUnitsCount); - return (RawErasureDecoder) rawCoder; - } - - /** - * Create raw encoder using the factory specified by rawCoderFactoryKey - * @param rawCoderFactoryKey - * @return raw encoder - */ - protected RawErasureEncoder createRawEncoder( - String rawCoderFactoryKey, int dataUnitsCount, int parityUnitsCount) { - RawErasureCoder rawCoder = createRawCoder(getConf(), - rawCoderFactoryKey, true, dataUnitsCount, parityUnitsCount); - return (RawErasureEncoder) rawCoder; - } - - /** - * Create raw coder using specified conf and raw coder factory key. - * @param conf - * @param rawCoderFactoryKey - * @param isEncoder - * @return raw coder - */ - public static RawErasureCoder createRawCoder(Configuration conf, - String rawCoderFactoryKey, boolean isEncoder, int numDataUnits, - int numParityUnits) { - - if (conf == null) { - return null; - } - - Class factClass = null; - factClass = conf.getClass(rawCoderFactoryKey, - factClass, RawErasureCoderFactory.class); - - if (factClass == null) { - return null; - } - - RawErasureCoderFactory fact; - try { - fact = factClass.newInstance(); - } catch (InstantiationException e) { - throw new RuntimeException("Failed to create raw coder", e); - } catch (IllegalAccessException e) { - throw new RuntimeException("Failed to create raw coder", e); - } - - return isEncoder ? fact.createEncoder(numDataUnits, numParityUnits) : - fact.createDecoder(numDataUnits, numParityUnits); - } - public AbstractErasureCoder(int numDataUnits, int numParityUnits) { this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; } public AbstractErasureCoder(ECSchema schema) { - this(schema.getNumDataUnits(), schema.getNumParityUnits()); + this(schema.getNumDataUnits(), schema.getNumParityUnits()); } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java index 57f4373ac1772..f56674d32c240 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -17,11 +17,10 @@ */ package org.apache.hadoop.io.erasurecode.coder; -import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; /** @@ -53,12 +52,8 @@ protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { private RawErasureDecoder checkCreateRSRawDecoder() { if (rsRawDecoder == null) { - rsRawDecoder = createRawDecoder( - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, - getNumDataUnits(), getNumParityUnits()); - if (rsRawDecoder == null) { - rsRawDecoder = new RSRawDecoder(getNumDataUnits(), getNumParityUnits()); - } + rsRawDecoder = CodecUtil.createRSRawDecoder(getConf(), + getNumDataUnits(), getNumParityUnits()); } return rsRawDecoder; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java index ab23474f54cdf..3ed3e2091d9ce 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java @@ -17,11 +17,10 @@ */ package org.apache.hadoop.io.erasurecode.coder; -import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; /** @@ -53,12 +52,8 @@ protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { private RawErasureEncoder checkCreateRSRawEncoder() { if (rawEncoder == null) { - rawEncoder = createRawEncoder( - CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + rawEncoder = CodecUtil.createRSRawEncoder(getConf(), getNumDataUnits(), getNumParityUnits()); - if (rawEncoder == null) { - rawEncoder = new RSRawEncoder(getNumDataUnits(), getNumParityUnits()); - } } return rawEncoder; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java index 3fe8d1bf82d64..a84741824e877 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.io.erasurecode.coder; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder; /** * Xor erasure decoder that decodes a block group. @@ -39,10 +39,10 @@ public XORErasureDecoder(ECSchema schema) { } @Override - protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { - // May be configured - RawErasureDecoder rawDecoder = new XORRawDecoder( - getNumDataUnits(), getNumParityUnits()); + protected ErasureCodingStep prepareDecodingStep( + final ECBlockGroup blockGroup) { + RawErasureDecoder rawDecoder = CodecUtil.createXORRawDecoder(getConf(), + getNumDataUnits(), getNumParityUnits()); ECBlock[] inputBlocks = getInputBlocks(blockGroup); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java index 5020896b66d46..5c4bcddb5150e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.io.erasurecode.coder; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECBlock; import org.apache.hadoop.io.erasurecode.ECBlockGroup; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; -import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder; /** * Xor erasure encoder that encodes a block group. @@ -39,10 +39,10 @@ public XORErasureEncoder(ECSchema schema) { } @Override - protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) { - // May be configured - RawErasureEncoder rawEncoder = new XORRawEncoder( - getNumDataUnits(), getNumParityUnits()); + protected ErasureCodingStep prepareEncodingStep( + final ECBlockGroup blockGroup) { + RawErasureEncoder rawEncoder = CodecUtil.createXORRawEncoder(getConf(), + getNumDataUnits(), getNumParityUnits()); ECBlock[] inputBlocks = getInputBlocks(blockGroup); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index bf99f17d7fd0e..a7339b7d16930 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -42,6 +42,7 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; @@ -155,7 +156,8 @@ boolean include(long pos) { curStripeRange = new StripeRange(0, 0); readingService = new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); - decoder = new RSRawDecoder(dataBlkNum, parityBlkNum); + decoder = CodecUtil.createRSRawDecoder(dfsClient.getConfiguration(), + dataBlkNum, parityBlkNum); if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Creating an striped input stream for file " + src); } @@ -207,8 +209,8 @@ private synchronized void blockSeekTo(long target) throws IOException { // The purpose is to get start offset into each block. long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema, cellSize, targetBlockGroup, offsetIntoBlockGroup); - Preconditions.checkState( - offsetsForInternalBlocks.length == dataBlkNum + parityBlkNum); + Preconditions.checkState(offsetsForInternalBlocks.length == + dataBlkNum + parityBlkNum); long minOffset = offsetsForInternalBlocks[dataBlkNum]; retry = new ReaderRetryPolicy(); @@ -726,8 +728,10 @@ void prepareDecodeInputs() { void prepareParityChunk() { for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { if (alignedStripe.chunks[i] == null) { - final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); - alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); + final int decodeIndex = convertIndex4Decode(i, + dataBlkNum, parityBlkNum); + alignedStripe.chunks[i] = + new StripingChunk(decodeInputs[decodeIndex]); alignedStripe.chunks[i].addByteArraySlice(0, (int) alignedStripe.getSpanInBlock()); break; @@ -807,7 +811,8 @@ void prepareParityChunk() throws IOException { parityBlkNum); decodeInputs[decodeIndex] = ByteBuffer.allocateDirect( (int) alignedStripe.range.spanInBlock); - alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); + alignedStripe.chunks[i] = + new StripingChunk(decodeInputs[decodeIndex]); if (blockReaders[i] == null) { prepareParityBlockReader(i); } @@ -839,7 +844,8 @@ void decode() { // decoders to work final int span = (int) alignedStripe.getSpanInBlock(); for (int i = 0; i < alignedStripe.chunks.length; i++) { - final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + final int decodeIndex = convertIndex4Decode(i, + dataBlkNum, parityBlkNum); if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state == StripingChunk.ALLZERO) { for (int j = 0; j < span; j++) { @@ -857,7 +863,8 @@ void decode() { for (int i = 0; i < alignedStripe.chunks.length; i++) { if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state == StripingChunk.MISSING) { - decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + decodeIndices[pos++] = convertIndex4Decode(i, + dataBlkNum, parityBlkNum); } } decodeIndices = Arrays.copyOf(decodeIndices, pos); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 0935d5c634a46..bdd3352179346 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.io.MultipleIOException; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; @@ -247,13 +248,16 @@ ExtendedBlock getBlock() { numDataBlocks = schema.getNumDataUnits(); numAllBlocks = numDataBlocks + numParityBlocks; - encoder = new RSRawEncoder(numDataBlocks, numParityBlocks); + encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(), + numDataBlocks, numParityBlocks); - coordinator = new Coordinator(dfsClient.getConf(), numDataBlocks, numAllBlocks); + coordinator = new Coordinator(dfsClient.getConf(), + numDataBlocks, numAllBlocks); try { cellBuffers = new CellBuffers(numParityBlocks); } catch (InterruptedException ie) { - throw DFSUtil.toInterruptedIOException("Failed to create cell buffers", ie); + throw DFSUtil.toInterruptedIOException( + "Failed to create cell buffers", ie); } List s = new ArrayList<>(numAllBlocks); @@ -318,7 +322,8 @@ private void checkStreamers() throws IOException { } } - private void handleStreamerFailure(String err, Exception e) throws IOException { + private void handleStreamerFailure(String err, + Exception e) throws IOException { LOG.warn("Failed: " + err + ", " + this, e); getCurrentStreamer().setIsFailed(true); checkStreamers(); @@ -487,7 +492,8 @@ private void writeParityCellsForLastStripe() throws IOException { return; } - final int firstCellSize = (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize); + final int firstCellSize = + (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize); final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize? firstCellSize : cellSize; final ByteBuffer[] buffers = cellBuffers.getBuffers(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 6f3857feee9ef..3c9adc449b2f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; @@ -112,7 +113,7 @@ public ErasureCodingWorker(Configuration conf, DataNode datanode) { } private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) { - return new RSRawDecoder(numDataUnits, numParityUnits); + return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits); } private void initializeStripedReadThreadPool(int num) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index b53983bc5a85e..b29d58238465b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -37,8 +37,10 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -230,8 +232,9 @@ public void testPreadWithDNFailure() throws Exception { for (int m : missingBlkIdx) { decodeInputs[m] = null; } - RSRawDecoder rsRawDecoder = new RSRawDecoder(DATA_BLK_NUM, PARITY_BLK_NUM); - rsRawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs); + RawErasureDecoder rawDecoder = CodecUtil.createRSRawDecoder(conf, + DATA_BLK_NUM, PARITY_BLK_NUM); + rawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs); int posInBuf = i * CELLSIZE * DATA_BLK_NUM + failedDNIdx * CELLSIZE; System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index e041dbe303e66..3f40deeae29a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.test.GenericTestUtils; @@ -43,7 +44,8 @@ import org.junit.Test; public class TestDFSStripedOutputStream { - public static final Log LOG = LogFactory.getLog(TestDFSStripedOutputStream.class); + public static final Log LOG = LogFactory.getLog( + TestDFSStripedOutputStream.class); static { GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); @@ -55,6 +57,7 @@ public class TestDFSStripedOutputStream { private MiniDFSCluster cluster; private DistributedFileSystem fs; + private Configuration conf; private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final int stripesPerBlock = 4; private final int blockSize = cellSize * stripesPerBlock; @@ -62,7 +65,7 @@ public class TestDFSStripedOutputStream { @Before public void setup() throws IOException { int numDNs = dataBlocks + parityBlocks + 2; - Configuration conf = new Configuration(); + conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0); @@ -140,7 +143,8 @@ public void testFileMoreThanABlockGroup1() throws IOException { @Test public void testFileMoreThanABlockGroup2() throws IOException { - testOneFile("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123); + testOneFile("/MoreThanABlockGroup2", + blockSize * dataBlocks + cellSize+ 123); } @@ -251,13 +255,14 @@ void checkData(String src, int writeBytes) throws IOException { } } - static void verifyParity(final long size, final int cellSize, + void verifyParity(final long size, final int cellSize, byte[][] dataBytes, byte[][] parityBytes) { - verifyParity(size, cellSize, dataBytes, parityBytes, -1); + verifyParity(conf, size, cellSize, dataBytes, parityBytes, -1); } - static void verifyParity(final long size, final int cellSize, - byte[][] dataBytes, byte[][] parityBytes, int killedDnIndex) { + static void verifyParity(Configuration conf, final long size, + final int cellSize, byte[][] dataBytes, + byte[][] parityBytes, int killedDnIndex) { // verify the parity blocks int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength( size, cellSize, dataBytes.length, dataBytes.length); @@ -275,7 +280,8 @@ static void verifyParity(final long size, final int cellSize, } } final RawErasureEncoder encoder = - new RSRawEncoder(dataBytes.length, parityBytes.length); + CodecUtil.createRSRawEncoder(conf, + dataBytes.length, parityBytes.length); encoder.encode(dataBytes, expectedParityBytes); for (int i = 0; i < parityBytes.length; i++) { if (i != killedDnIndex) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index c232e13610180..d2e045827188c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -335,7 +335,7 @@ static void checkData(DistributedFileSystem dfs, String src, int length, } // check parity - TestDFSStripedOutputStream.verifyParity( + TestDFSStripedOutputStream.verifyParity(dfs.getConf(), lbs.getLocatedBlocks().get(group).getBlockSize(), CELL_SIZE, dataBlockBytes, parityBlockBytes, killedDnIndex - dataBlockBytes.length); From c5d4652dedc1e4f8908117fcfc4872c3efd67b3e Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Wed, 10 Jun 2015 10:42:45 +0530 Subject: [PATCH 154/212] HDFS-8556. Erasure Coding: Fix usage of 'createZone' (Contributed by Vinayakumar B) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 ++ .../org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 61d7a763af534..2118a0c4193aa 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -289,3 +289,5 @@ HDFS-8460. Erasure Coding: stateful read result doesn't match data occasionally because of flawed test. (Walter Su via zhz) + + HDFS-8556. Erasure Coding: Fix usage of 'createZone' (vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index 34965d29c7c0d..03026d895e4ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -83,7 +83,7 @@ protected void processPath(PathData item) throws IOException { */ static class CreateECZoneCommand extends ECCommand { public static final String NAME = "createZone"; - public static final String USAGE = "[-s ] "; + public static final String USAGE = "[-s ] [-c ] "; public static final String DESCRIPTION = "Create a zone to encode files using a specified schema\n" + "Options :\n" From b7d6ea8e91417a23e8c07c3db977277b485a2fdd Mon Sep 17 00:00:00 2001 From: Walter Su Date: Thu, 11 Jun 2015 10:26:07 +0800 Subject: [PATCH 155/212] HDFS-8571. Fix TestErasureCodingCli test. Contributed by Vinayakumar B. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 ++ .../hadoop-hdfs/src/test/resources/testErasureCodingConf.xml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2118a0c4193aa..5e66df0805549 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -291,3 +291,5 @@ occasionally because of flawed test. (Walter Su via zhz) HDFS-8556. Erasure Coding: Fix usage of 'createZone' (vinayakumarb) + + HDFS-8571. Fix TestErasureCodingCli test (Vinayakumar B via waltersu4549) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml index ee1a19aec2d49..70020d54db494 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -61,7 +61,7 @@ RegexpComparator - ^-createZone \[-s <schemaName>\] <path>(.)* + ^-createZone \[-s <schemaName>\] \[-c <cellSize>\] <path>(.)* From 98d340745be682fb251677bb4830aca76119868f Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Thu, 11 Jun 2015 10:48:02 +0530 Subject: [PATCH 156/212] HDFS-8450. Erasure Coding: Consolidate erasure coding zone related implementation into a single class (Contributed by Rakesh R) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../namenode/ErasureCodingZoneManager.java | 34 +-- .../server/namenode/FSDirErasureCodingOp.java | 217 ++++++++++++++++++ .../namenode/FSDirStatAndListingOp.java | 8 +- .../server/namenode/FSDirWriteFileOp.java | 10 +- .../hdfs/server/namenode/FSDirectory.java | 34 --- .../hdfs/server/namenode/FSEditLogLoader.java | 32 +-- .../hdfs/server/namenode/FSNamesystem.java | 77 +++---- .../server/namenode/NameNodeRpcServer.java | 2 +- .../hdfs/server/namenode/NamenodeFsck.java | 14 +- .../hdfs/server/namenode/NameNodeAdapter.java | 9 +- 11 files changed, 312 insertions(+), 128 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 5e66df0805549..fa39d72f42725 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -293,3 +293,6 @@ HDFS-8556. Erasure Coding: Fix usage of 'createZone' (vinayakumarb) HDFS-8571. Fix TestErasureCodingCli test (Vinayakumar B via waltersu4549) + + HDFS-8450. Erasure Coding: Consolidate erasure coding zone related + implementation into a single class (Rakesh R via vinayakumarb) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index e853829be56ab..263812622c107 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -60,14 +60,14 @@ public ErasureCodingZoneManager(FSDirectory dir) { this.dir = dir; } - ECSchema getECSchema(INodesInPath iip) throws IOException { - ErasureCodingZone ecZone = getECZone(iip); + ECSchema getErasureCodingSchema(INodesInPath iip) throws IOException { + ErasureCodingZone ecZone = getErasureCodingZone(iip); return ecZone == null ? null : ecZone.getSchema(); } - ErasureCodingZone getECZone(INodesInPath iip) throws IOException { + ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException { assert dir.hasReadLock(); - Preconditions.checkNotNull(iip); + Preconditions.checkNotNull(iip, "INodes cannot be null"); List inodes = iip.getReadOnlyINodes(); for (int i = inodes.size() - 1; i >= 0; i--) { final INode inode = inodes.get(i); @@ -90,8 +90,8 @@ ErasureCodingZone getECZone(INodesInPath iip) throws IOException { DataInputStream dIn=new DataInputStream(bIn); int cellSize = WritableUtils.readVInt(dIn); String schemaName = WritableUtils.readString(dIn); - ECSchema schema = dir.getFSNamesystem().getECSchemaManager() - .getSchema(schemaName); + ECSchema schema = dir.getFSNamesystem() + .getErasureCodingSchemaManager().getSchema(schemaName); return new ErasureCodingZone(dir.getInode(inode.getId()) .getFullPathName(), schema, cellSize); } @@ -100,22 +100,22 @@ ErasureCodingZone getECZone(INodesInPath iip) throws IOException { return null; } - XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize) - throws IOException { + List createErasureCodingZone(final INodesInPath srcIIP, + ECSchema schema, int cellSize) throws IOException { assert dir.hasWriteLock(); - final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false); + Preconditions.checkNotNull(srcIIP, "INodes cannot be null"); + String src = srcIIP.getPath(); if (dir.isNonEmptyDirectory(srcIIP)) { throw new IOException( "Attempt to create an erasure coding zone for a " + - "non-empty directory."); + "non-empty directory " + src); } - if (srcIIP != null && - srcIIP.getLastINode() != null && + if (srcIIP.getLastINode() != null && !srcIIP.getLastINode().isDirectory()) { throw new IOException("Attempt to create an erasure coding zone " + - "for a file."); + "for a file " + src); } - if (getECSchema(srcIIP) != null) { + if (getErasureCodingSchema(srcIIP) != null) { throw new IOException("Directory " + src + " is already in an " + "erasure coding zone."); } @@ -147,14 +147,14 @@ XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize) xattrs.add(ecXAttr); FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs, EnumSet.of(XAttrSetFlag.CREATE)); - return ecXAttr; + return xattrs; } void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src) throws IOException { assert dir.hasReadLock(); - final ErasureCodingZone srcZone = getECZone(srcIIP); - final ErasureCodingZone dstZone = getECZone(dstIIP); + final ErasureCodingZone srcZone = getErasureCodingZone(srcIIP); + final ErasureCodingZone dstZone = getErasureCodingZone(dstIIP); if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) { return; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java new file mode 100644 index 0000000000000..fd7ef333cdb4c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java @@ -0,0 +1,217 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * Helper class to perform erasure coding related operations. + */ +final class FSDirErasureCodingOp { + + /** + * Private constructor for preventing FSDirErasureCodingOp object + * creation. Static-only class. + */ + private FSDirErasureCodingOp() {} + + /** + * Create an erasure coding zone on directory src. + * + * @param fsn namespace + * @param srcArg the path of a directory which will be the root of the + * erasure coding zone. The directory must be empty. + * @param schema ECSchema for the erasure coding zone + * @param cellSize Cell size of stripe + * @param logRetryCache whether to record RPC ids in editlog for retry + * cache rebuilding + * @return {@link HdfsFileStatus} + * @throws IOException + */ + static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn, + final String srcArg, final ECSchema schema, final int cellSize, + final boolean logRetryCache) throws IOException { + assert fsn.hasWriteLock(); + + String src = srcArg; + FSPermissionChecker pc = null; + byte[][] pathComponents = null; + pathComponents = FSDirectory.getPathComponentsForReservedPath(src); + pc = fsn.getPermissionChecker(); + FSDirectory fsd = fsn.getFSDirectory(); + src = fsd.resolvePath(pc, src, pathComponents); + final INodesInPath iip; + List xAttrs; + fsd.writeLock(); + try { + iip = fsd.getINodesInPath4Write(src, false); + xAttrs = fsn.getErasureCodingZoneManager().createErasureCodingZone( + iip, schema, cellSize); + } finally { + fsd.writeUnlock(); + } + fsn.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); + return fsd.getAuditFileInfo(iip); + } + + /** + * Get the erasure coding zone information for specified path. + * + * @param fsn namespace + * @param src path + * @return {@link ErasureCodingZone} + * @throws IOException + */ + static ErasureCodingZone getErasureCodingZone(final FSNamesystem fsn, + final String src) throws IOException { + assert fsn.hasReadLock(); + + final INodesInPath iip = getINodesInPath(fsn, src); + return getErasureCodingZoneForPath(fsn, iip); + } + + /** + * Get erasure coding zone information for specified path. + * + * @param fsn namespace + * @param iip inodes in the path containing the file + * @return {@link ErasureCodingZone} + * @throws IOException + */ + static ErasureCodingZone getErasureCodingZone(final FSNamesystem fsn, + final INodesInPath iip) throws IOException { + assert fsn.hasReadLock(); + + return getErasureCodingZoneForPath(fsn, iip); + } + + /** + * Check if the file is in erasure coding zone. + * + * @param fsn namespace + * @param srcArg path + * @return true represents the file is in erasure coding zone, false otw + * @throws IOException + */ + static boolean isInErasureCodingZone(final FSNamesystem fsn, + final String srcArg) throws IOException { + assert fsn.hasReadLock(); + + final INodesInPath iip = getINodesInPath(fsn, srcArg); + return getErasureCodingSchemaForPath(fsn, iip) != null; + } + + /** + * Check if the file is in erasure coding zone. + * + * @param fsn namespace + * @param iip inodes in the path containing the file + * @return true represents the file is in erasure coding zone, false otw + * @throws IOException + */ + static boolean isInErasureCodingZone(final FSNamesystem fsn, + final INodesInPath iip) throws IOException { + return getErasureCodingSchema(fsn, iip) != null; + } + + /** + * Get erasure coding schema. + * + * @param fsn namespace + * @param iip inodes in the path containing the file + * @return {@link ECSchema} + * @throws IOException + */ + static ECSchema getErasureCodingSchema(final FSNamesystem fsn, + final INodesInPath iip) throws IOException { + assert fsn.hasReadLock(); + + return getErasureCodingSchemaForPath(fsn, iip); + } + + /** + * Get available erasure coding schemas. + * + * @param fsn namespace + * @return {@link ECSchema} array + */ + static ECSchema[] getErasureCodingSchemas(final FSNamesystem fsn) + throws IOException { + assert fsn.hasReadLock(); + + return fsn.getErasureCodingSchemaManager().getSchemas(); + } + + /** + * Get the ECSchema specified by the name. + * + * @param fsn namespace + * @param schemaName schema name + * @return {@link ECSchema} + */ + static ECSchema getErasureCodingSchema(final FSNamesystem fsn, + final String schemaName) throws IOException { + assert fsn.hasReadLock(); + + return fsn.getErasureCodingSchemaManager().getSchema(schemaName); + } + + private static INodesInPath getINodesInPath(final FSNamesystem fsn, + final String srcArg) throws IOException { + String src = srcArg; + final byte[][] pathComponents = FSDirectory + .getPathComponentsForReservedPath(src); + final FSDirectory fsd = fsn.getFSDirectory(); + final FSPermissionChecker pc = fsn.getPermissionChecker(); + src = fsd.resolvePath(pc, src, pathComponents); + INodesInPath iip = fsd.getINodesInPath(src, true); + if (fsn.isPermissionEnabled()) { + fsn.getFSDirectory().checkPathAccess(pc, iip, FsAction.READ); + } + return iip; + } + + private static ErasureCodingZone getErasureCodingZoneForPath( + final FSNamesystem fsn, final INodesInPath iip) throws IOException { + final FSDirectory fsd = fsn.getFSDirectory(); + fsd.readLock(); + try { + return fsn.getErasureCodingZoneManager().getErasureCodingZone(iip); + } finally { + fsd.readUnlock(); + } + } + + private static ECSchema getErasureCodingSchemaForPath(final FSNamesystem fsn, + final INodesInPath iip) throws IOException { + final FSDirectory fsd = fsn.getFSDirectory(); + fsd.readLock(); + try { + return fsn.getErasureCodingZoneManager().getErasureCodingSchema(iip); + } finally { + fsd.readUnlock(); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index 1b1d79ef480df..b18c2a37c7681 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -385,8 +385,9 @@ static HdfsFileStatus createFileStatus( final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); - - final ErasureCodingZone ecZone = fsd.getECZone(iip); + + final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + fsd.getFSNamesystem(), iip); final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; @@ -468,7 +469,8 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - final ErasureCodingZone ecZone = fsd.getECZone(iip); + final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + fsd.getFSNamesystem(), iip); final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index c037cd74a98c7..a3daabcc59da7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -495,7 +495,8 @@ static INodeFile addFileForEditLog( INodesInPath iip = fsd.addINode(existing, newNode); if (iip != null) { // check if the file is in an EC zone - if (fsd.isInECZone(iip)) { + if (FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(), + iip)) { newNode.addStripedBlocksFeature(); } if (aclEntries != null) { @@ -530,7 +531,8 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, // associate new last block for the file final BlockInfo blockInfo; if (isStriped) { - ECSchema ecSchema = fsd.getECSchema(inodesInPath); + ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + fsd.getFSNamesystem(), inodesInPath); short numDataUnits = (short) ecSchema.getNumDataUnits(); short numParityUnits = (short) ecSchema.getNumParityUnits(); short numLocations = (short) (numDataUnits + numParityUnits); @@ -586,7 +588,9 @@ private static INodesInPath addFile( fsd.writeLock(); try { newiip = fsd.addINode(existing, newNode); - if (newiip != null && fsd.isInECZone(newiip)) { + if (newiip != null + && FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(), + newiip)) { newNode.addStripedBlocksFeature(); } } finally { 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 32c8e056af48c..aa49eb306ad6f 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 @@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException; import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException; @@ -58,7 +57,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.util.ByteArray; import org.apache.hadoop.hdfs.util.EnumCounters; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; @@ -1225,38 +1223,6 @@ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId, } } - XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize) - throws IOException { - writeLock(); - try { - return ecZoneManager.createErasureCodingZone(src, schema, cellSize); - } finally { - writeUnlock(); - } - } - - public boolean isInECZone(INodesInPath iip) throws IOException { - return getECSchema(iip) != null; - } - - ECSchema getECSchema(INodesInPath iip) throws IOException { - readLock(); - try { - return ecZoneManager.getECSchema(iip); - } finally { - readUnlock(); - } - } - - ErasureCodingZone getECZone(INodesInPath iip) throws IOException { - readLock(); - try { - return ecZoneManager.getECZone(iip); - } finally { - readUnlock(); - } - } - static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException { INode inode = iip.getLastINode(); if (inode == null) { 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 66b2f82edf23e..62a278cd5a39e 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 @@ -417,8 +417,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - updateBlocks(fsDir, addCloseOp, iip, newFile, - fsDir.getECSchema(iip), fsDir.isInECZone(iip)); + ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + fsDir.getFSNamesystem(), iip); + updateBlocks(fsDir, addCloseOp, iip, newFile, ecSchema); break; } case OP_CLOSE: { @@ -438,8 +439,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - updateBlocks(fsDir, addCloseOp, iip, file, - fsDir.getECSchema(iip), fsDir.isInECZone(iip)); + ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + fsDir.getFSNamesystem(), iip); + updateBlocks(fsDir, addCloseOp, iip, file, ecSchema); // Now close the file if (!file.isUnderConstruction() && @@ -497,8 +499,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures - updateBlocks(fsDir, updateOp, iip, oldFile, - fsDir.getECSchema(iip), fsDir.isInECZone(iip)); + ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + fsDir.getFSNamesystem(), iip); + updateBlocks(fsDir, updateOp, iip, oldFile, ecSchema); if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); @@ -515,8 +518,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // add the new block to the INodeFile - addNewBlock(addBlockOp, oldFile, - fsDir.getECSchema(iip), fsDir.isInECZone(iip)); + ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + fsDir.getFSNamesystem(), iip); + addNewBlock(addBlockOp, oldFile, ecSchema); break; } case OP_SET_REPLICATION: { @@ -957,8 +961,7 @@ private static String formatEditLogReplayError(EditLogInputStream in, /** * Add a new block into the given INodeFile */ - private void addNewBlock(AddBlockOp op, INodeFile file, - ECSchema schema, boolean isStriped) + private void addNewBlock(AddBlockOp op, INodeFile file, ECSchema ecSchema) throws IOException { BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); @@ -986,8 +989,9 @@ private void addNewBlock(AddBlockOp op, INodeFile file, } // add the new block final BlockInfo newBlockInfo; + boolean isStriped = ecSchema != null; if (isStriped) { - newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, schema); + newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, ecSchema); } else { newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); @@ -1002,8 +1006,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file, * @throws IOException */ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, - INodesInPath iip, INodeFile file, ECSchema schema, - boolean isStriped) throws IOException { + INodesInPath iip, INodeFile file, ECSchema ecSchema) throws IOException { // Update its block list BlockInfo[] oldBlocks = file.getBlocks(); Block[] newBlocks = op.getBlocks(); @@ -1062,6 +1065,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, throw new IOException("Trying to delete non-existant block " + oldBlock); } } else if (newBlocks.length > oldBlocks.length) { + final boolean isStriped = ecSchema != null; // We're adding blocks for (int i = oldBlocks.length; i < newBlocks.length; i++) { Block newBlock = newBlocks[i]; @@ -1071,7 +1075,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // what about an old-version fsync() where fsync isn't called // until several blocks in? if (isStriped) { - newBI = new BlockInfoStripedUnderConstruction(newBlock, schema); + newBI = new BlockInfoStripedUnderConstruction(newBlock, ecSchema); } else { newBI = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); 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 59c9a03fbe919..655ead891895f 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 @@ -2408,7 +2408,7 @@ private HdfsFileStatus startFileInt(final String src, readLock(); try { checkOperation(OperationCategory.READ); - if (!isInECZone(src)) { + if (!FSDirErasureCodingOp.isInErasureCodingZone(this, src)) { blockManager.verifyReplication(src, replication, clientMachine); } } finally { @@ -3675,7 +3675,8 @@ void commitOrCompleteLastBlock( final long diff; final short replicationFactor; if (fileINode.isStriped()) { - final ECSchema ecSchema = dir.getECSchema(iip); + final ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + this, iip); final short numDataUnits = (short) ecSchema.getNumDataUnits(); final short numParityUnits = (short) ecSchema.getNumParityUnits(); @@ -6670,11 +6671,16 @@ public CacheManager getCacheManager() { return cacheManager; } - /** @return the schema manager. */ - public ErasureCodingSchemaManager getECSchemaManager() { + /** @return the ErasureCodingSchemaManager. */ + public ErasureCodingSchemaManager getErasureCodingSchemaManager() { return ecSchemaManager; } + /** @return the ErasureCodingZoneManager. */ + public ErasureCodingZoneManager getErasureCodingZoneManager() { + return dir.ecZoneManager; + } + @Override // NameNodeMXBean public String getCorruptFiles() { List list = new ArrayList(); @@ -7579,47 +7585,25 @@ BatchedListEntries listEncryptionZones(long prevId) void createErasureCodingZone(final String srcArg, final ECSchema schema, int cellSize, final boolean logRetryCache) throws IOException, UnresolvedLinkException, SafeModeException, AccessControlException { - String src = srcArg; + checkSuperuserPrivilege(); + checkOperation(OperationCategory.WRITE); HdfsFileStatus resultingStat = null; - FSPermissionChecker pc = null; - byte[][] pathComponents = null; boolean success = false; - try { - checkSuperuserPrivilege(); - checkOperation(OperationCategory.WRITE); - pathComponents = - FSDirectory.getPathComponentsForReservedPath(src); - pc = getPermissionChecker(); - } catch (Throwable e) { - logAuditEvent(success, "createErasureCodingZone", srcArg); - throw e; - } writeLock(); try { - checkSuperuserPrivilege(); checkOperation(OperationCategory.WRITE); - checkNameNodeSafeMode("Cannot create erasure coding zone on " + src); - src = dir.resolvePath(pc, src, pathComponents); - - final XAttr ecXAttr = dir.createErasureCodingZone(src, schema, cellSize); - List xAttrs = Lists.newArrayListWithCapacity(1); - xAttrs.add(ecXAttr); - getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); - final INodesInPath iip = dir.getINodesInPath4Write(src, false); - resultingStat = dir.getAuditFileInfo(iip); + checkNameNodeSafeMode("Cannot create erasure coding zone on " + srcArg); + resultingStat = FSDirErasureCodingOp.createErasureCodingZone(this, + srcArg, schema, cellSize, logRetryCache); success = true; } finally { writeUnlock(); + if (success) { + getEditLog().logSync(); + } + logAuditEvent(success, "createErasureCodingZone", srcArg, null, + resultingStat); } - getEditLog().logSync(); - logAuditEvent(success, "createErasureCodingZone", srcArg, null, resultingStat); - } - - private boolean isInECZone(String src) throws IOException { - byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); - src = FSDirectory.resolvePath(src, pathComponents, dir); - final INodesInPath iip = dir.getINodesInPath(src, true); - return dir.isInECZone(iip); } /** @@ -7638,15 +7622,15 @@ ErasureCodingZone getErasureCodingZone(String src) } /** - * Get available ECSchemas + * Get available erasure coding schemas */ - ECSchema[] getECSchemas() throws IOException { + ECSchema[] getErasureCodingSchemas() throws IOException { checkOperation(OperationCategory.READ); waitForLoadingFSImage(); readLock(); try { checkOperation(OperationCategory.READ); - return ecSchemaManager.getSchemas(); + return FSDirErasureCodingOp.getErasureCodingSchemas(this); } finally { readUnlock(); } @@ -7655,13 +7639,13 @@ ECSchema[] getECSchemas() throws IOException { /** * Get the ECSchema specified by the name */ - ECSchema getECSchema(String schemaName) throws IOException { + ECSchema getErasureCodingSchema(String schemaName) throws IOException { checkOperation(OperationCategory.READ); waitForLoadingFSImage(); readLock(); try { checkOperation(OperationCategory.READ); - return ecSchemaManager.getSchema(schemaName); + return FSDirErasureCodingOp.getErasureCodingSchema(this, schemaName); } finally { readUnlock(); } @@ -7854,16 +7838,7 @@ private static void enableAsyncAuditLog() { @Override public ErasureCodingZone getErasureCodingZoneForPath(String src) throws IOException { - final byte[][] pathComponents = FSDirectory - .getPathComponentsForReservedPath(src); - final FSPermissionChecker pc = getPermissionChecker(); - src = dir.resolvePath(pc, src, pathComponents); - final INodesInPath iip = dir.getINodesInPath(src, true); - if (isPermissionEnabled) { - dir.checkPathAccess(pc, iip, FsAction.READ); - } - return dir.getECZone(iip); + return FSDirErasureCodingOp.getErasureCodingZone(this, src); } - } 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 1377bbed4f920..031ae085a72d9 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 @@ -2037,7 +2037,7 @@ public void removeSpanReceiver(long id) throws IOException { @Override // ClientProtocol public ECSchema[] getECSchemas() throws IOException { checkNNStartup(); - return namesystem.getECSchemas(); + return namesystem.getErasureCodingSchemas(); } @Override // ClientProtocol 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 1c60a884dcaa3..0737502ea5380 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 @@ -572,9 +572,17 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res // count expected replicas short targetFileReplication; if(file.getReplication() == 0) { - INode inode = namenode.getNamesystem().getFSDirectory().getINode(path); - INodesInPath iip = INodesInPath.fromINode(inode); - ECSchema ecSchema = namenode.getNamesystem().getFSDirectory().getECSchema(iip); + final FSNamesystem fsn = namenode.getNamesystem(); + final ECSchema ecSchema; + fsn.readLock(); + try { + INode inode = namenode.getNamesystem().getFSDirectory() + .getINode(path); + INodesInPath iip = INodesInPath.fromINode(inode); + ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(fsn, iip); + } finally { + fsn.readUnlock(); + } targetFileReplication = (short) (ecSchema.getNumDataUnits() + ecSchema.getNumParityUnits()); } else { targetFileReplication = file.getReplication(); 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 4ca5edaf164fb..975c4f25ca152 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 @@ -71,8 +71,13 @@ public static LocatedBlocks getBlockLocations(NameNode namenode, public static HdfsFileStatus getFileInfo(NameNode namenode, String src, boolean resolveLink) throws AccessControlException, UnresolvedLinkException, StandbyException, IOException { - return FSDirStatAndListingOp.getFileInfo(namenode.getNamesystem() - .getFSDirectory(), src, resolveLink); + namenode.getNamesystem().readLock(); + try { + return FSDirStatAndListingOp.getFileInfo(namenode.getNamesystem() + .getFSDirectory(), src, resolveLink); + } finally { + namenode.getNamesystem().readUnlock(); + } } public static boolean mkdirs(NameNode namenode, String src, From 683332b36de1040eb8901d676e666527e8c5f8fe Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 12 Jun 2015 14:48:53 -0700 Subject: [PATCH 157/212] HDFS-8585. Erasure Coding: Remove dataBlockNum and parityBlockNum from StripedBlockProto. Contributed by Yi Liu. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hadoop/hdfs/protocolPB/PBHelper.java | 16 ------------- .../server/namenode/FSImageFormatPBINode.java | 23 ++++++++++--------- .../offlineImageViewer/FSImageLoader.java | 20 ++-------------- .../hadoop-hdfs/src/main/proto/fsimage.proto | 2 +- .../hadoop-hdfs/src/main/proto/hdfs.proto | 10 -------- ...stOfflineImageViewerWithStripedBlocks.java | 14 ++++------- 7 files changed, 23 insertions(+), 65 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index fa39d72f42725..2eb82590be059 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -296,3 +296,6 @@ HDFS-8450. Erasure Coding: Consolidate erasure coding zone related implementation into a single class (Rakesh R via vinayakumarb) + + HDFS-8585. Erasure Coding: Remove dataBlockNum and parityBlockNum from + StripedBlockProto. (Yi Liu via jing9) 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 0bfc3bbf3abcb..7ee6112506b1d 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 @@ -183,7 +183,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto; @@ -195,7 +194,6 @@ 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.BlockInfoStriped; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; @@ -444,20 +442,6 @@ public static Block convert(BlockProto b) { return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp()); } - public static BlockInfoStriped convert(StripedBlockProto p, ECSchema schema) { - return new BlockInfoStriped(convert(p.getBlock()), schema); - } - - public static StripedBlockProto convert(BlockInfoStriped blk) { - BlockProto bp = BlockProto.newBuilder().setBlockId(blk.getBlockId()) - .setGenStamp(blk.getGenerationStamp()).setNumBytes(blk.getNumBytes()) - .build(); - return StripedBlockProto.newBuilder() - .setDataBlockNum(blk.getDataBlockNum()) - .setParityBlockNum(blk.getParityBlockNum()) - .setBlock(bp).build(); - } - public static BlockWithLocationsProto convert(BlockWithLocations blk) { BlockWithLocationsProto.Builder builder = BlockWithLocationsProto .newBuilder().setBlock(convert(blk.getBlock())) 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 e157b95c52198..92e705f30a506 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.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; @@ -42,7 +41,6 @@ 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.protocol.proto.HdfsProtos.StripedBlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; @@ -330,10 +328,14 @@ private INodeFile loadINodeFile(INodeSection.INode n) { short replication = (short) f.getReplication(); LoaderContext state = parent.getLoaderContext(); - BlockInfoContiguous[] blocks = new BlockInfoContiguous[bp.size()]; - for (int i = 0, e = bp.size(); i < e; ++i) { - blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication); + BlockInfoContiguous[] blocks = null; + if (!f.hasStripedBlocks()) { + blocks = new BlockInfoContiguous[bp.size()]; + for (int i = 0, e = bp.size(); i < e; ++i) { + blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication); + } } + final PermissionStatus permissions = loadPermission(f.getPermission(), parent.getLoaderContext().getStringTable()); @@ -357,10 +359,9 @@ private INodeFile loadINodeFile(INodeSection.INode n) { if (f.hasStripedBlocks()) { // TODO: HDFS-7859 ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); - StripedBlocksFeature sb = f.getStripedBlocks(); stripeFeature = file.addStripedBlocksFeature(); - for (StripedBlockProto sp : sb.getBlocksList()) { - stripeFeature.addBlock(PBHelper.convert(sp, schema)); + for (BlockProto b : bp) { + stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b), schema)); } } @@ -658,14 +659,14 @@ private void save(OutputStream out, INodeFile n) throws IOException { FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature(); if (sb != null) { - StripedBlocksFeature.Builder builder = - StripedBlocksFeature.newBuilder(); BlockInfoStriped[] sblocks = sb.getBlocks(); if (sblocks != null) { for (BlockInfoStriped sblk : sblocks) { - builder.addBlocks(PBHelper.convert(sblk)); + b.addBlocks(PBHelper.convert(sblk)); } } + StripedBlocksFeature.Builder builder = + StripedBlocksFeature.newBuilder(); b.setStripedBlocks(builder.build()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java index 42f6c0be27d3d..25d7722cb664d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java @@ -41,15 +41,12 @@ import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf; import org.apache.hadoop.hdfs.server.namenode.FSImageUtil; import org.apache.hadoop.hdfs.server.namenode.FsImageProto; import org.apache.hadoop.hdfs.server.namenode.INodeId; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.hdfs.web.JsonUtil; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.LimitInputStream; @@ -485,21 +482,8 @@ private long lookup(String path) throws IOException { static long getFileSize(FsImageProto.INodeSection.INodeFile f) { long size = 0; - if (f.hasStripedBlocks()) { - List blocksList = f.getStripedBlocks().getBlocksList(); - // Get total of actual data block size - for (StripedBlockProto p : blocksList) { - // Total usage by this striped blocks should be the total of data - // blocks and parity blocks - size += StripedBlockUtil.spaceConsumedByStripedBlock(p.getBlock() - .getNumBytes(), p.getDataBlockNum(), p.getParityBlockNum(), - HdfsConstants.BLOCK_STRIPED_CELL_SIZE); - } - } else { - for (HdfsProtos.BlockProto p : f.getBlocksList()) { - size += p.getNumBytes(); - } - + for (HdfsProtos.BlockProto p : f.getBlocksList()) { + size += p.getNumBytes(); } return size; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto index 3f3a71ec64f88..4d0e2ddea89ff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto @@ -93,7 +93,7 @@ message INodeSection { } message StripedBlocksFeature { - repeated StripedBlockProto blocks = 1; + // store striped blocks related information } message AclFeatureProto { 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 e6db596bc932b..dd48d7fdb5e86 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -514,16 +514,6 @@ message BlockProto { optional uint64 numBytes = 3 [default = 0]; } -/** - * Striped block information. Besides the basic information for a block, - * it also contains the number of data/parity blocks. - */ -message StripedBlockProto { - required BlockProto block = 1; - optional uint32 dataBlockNum = 2; - optional uint32 parityBlockNum = 3; -} - /** * Block and datanodes where is it located */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java index 2a51f99b79e9d..f4efbcf0d1f82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; import org.apache.hadoop.hdfs.server.namenode.INodeFile; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -136,28 +135,25 @@ private void testFileSize(int numBytes) throws IOException, } FSImageLoader loader = FSImageLoader.load(orgFsimage.getAbsolutePath()); String fileStatus = loader.getFileStatus("/eczone/striped"); - long expectedSpaceConsumed = StripedBlockUtil.spaceConsumedByStripedBlock( - bytes.length, HdfsConstants.NUM_DATA_BLOCKS, - HdfsConstants.NUM_PARITY_BLOCKS, HdfsConstants.BLOCK_STRIPED_CELL_SIZE); + long expectedFileSize = bytes.length; // Verify space consumed present in BlockInfoStriped FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); assertTrue("Invalid block size", fileNode.getBlocks().length > 0); - long actualSpaceConsumed = 0; + long actualFileSize = 0; for (BlockInfo blockInfo : fileNode.getBlocks()) { assertTrue("Didn't find block striped information", blockInfo instanceof BlockInfoStriped); - BlockInfoStriped b = (BlockInfoStriped) blockInfo; - actualSpaceConsumed += b.spaceConsumed(); + actualFileSize += blockInfo.getNumBytes(); } assertEquals("Wrongly computed file size contains striped blocks", - expectedSpaceConsumed, actualSpaceConsumed); + expectedFileSize, actualFileSize); // Verify space consumed present in filestatus String EXPECTED_FILE_SIZE = "\"length\":" - + String.valueOf(expectedSpaceConsumed); + + String.valueOf(expectedFileSize); assertTrue( "Wrongly computed file size contains striped blocks, file status:" + fileStatus + ". Expected file size is : " + EXPECTED_FILE_SIZE, From 49d5cff49011cc0878665204e22b5c832bc914ce Mon Sep 17 00:00:00 2001 From: yliu Date: Sun, 14 Jun 2015 15:39:19 +0800 Subject: [PATCH 158/212] HDFS-8559. Erasure Coding: fix non-protobuf fsimage for striped blocks. (Jing Zhao via yliu) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hdfs/server/namenode/FSImageFormat.java | 58 +++----------- .../server/namenode/FSImageSerialization.java | 76 +++++-------------- .../hdfs/server/namenode/TestFSImage.java | 22 +----- 4 files changed, 33 insertions(+), 126 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2eb82590be059..1ae3e9b3fad87 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -299,3 +299,6 @@ HDFS-8585. Erasure Coding: Remove dataBlockNum and parityBlockNum from StripedBlockProto. (Yi Liu via jing9) + + HDFS-8559. Erasure Coding: fix non-protobuf fsimage for striped blocks. + (Jing Zhao via yliu) 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 3083952682036..d9a74e6fd5303 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 @@ -365,12 +365,6 @@ public void load(File curFile) throws IOException { long maxSequentialBlockId = in.readLong(); namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId( maxSequentialBlockId); - if (NameNodeLayoutVersion.supports( - NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)) { - final long maxStripedBlockId = in.readLong(); - namesystem.getBlockIdManager().setLastAllocatedStripedBlockId( - maxStripedBlockId); - } } else { long startingGenStamp = namesystem.getBlockIdManager() @@ -759,31 +753,16 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, atime = in.readLong(); } final long blockSize = in.readLong(); - final boolean isStriped = NameNodeLayoutVersion.supports( - NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion) - && (in.readBoolean()); final int numBlocks = in.readInt(); - // TODO: ECSchema can be restored from persisted file (HDFS-7859). - final ECSchema schema = isStriped ? - ErasureCodingSchemaManager.getSystemDefaultSchema() : null; if (numBlocks >= 0) { // file // read blocks - Block[] blocks; - if (isStriped) { - blocks = new Block[numBlocks]; - for (int j = 0; j < numBlocks; j++) { - blocks[j] = new BlockInfoStriped(new Block(), schema); - blocks[j].readFields(in); - } - } else { - blocks = new BlockInfoContiguous[numBlocks]; - for (int j = 0; j < numBlocks; j++) { - blocks[j] = new BlockInfoContiguous(replication); - blocks[j].readFields(in); - } + Block[] blocks = new BlockInfoContiguous[numBlocks]; + for (int j = 0; j < numBlocks; j++) { + blocks[j] = new BlockInfoContiguous(replication); + blocks[j].readFields(in); } String clientName = ""; @@ -803,16 +782,8 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, // convert the last block to BlockUC if (blocks.length > 0) { Block lastBlk = blocks[blocks.length - 1]; - if (isStriped){ - BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk; - blocks[blocks.length - 1] - = new BlockInfoStripedUnderConstruction(lastBlk, - lastStripedBlk.getSchema()); - } else { - blocks[blocks.length - 1] - = new BlockInfoContiguousUnderConstruction(lastBlk, - replication); - } + blocks[blocks.length - 1] = + new BlockInfoContiguousUnderConstruction(lastBlk, replication); } } } @@ -825,19 +796,9 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, counter.increment(); } - INodeFile file; - if (isStriped) { - file = new INodeFile(inodeId, localName, permissions, modificationTime, - atime, new BlockInfoContiguous[0], (short) 0, blockSize); - file.addStripedBlocksFeature(); - for (Block block : blocks) { - file.getStripedBlocksFeature().addBlock((BlockInfoStriped) block); - } - } else { - file = new INodeFile(inodeId, localName, permissions, - modificationTime, atime, (BlockInfoContiguous[]) blocks, - replication, blockSize); - } + INodeFile file = new INodeFile(inodeId, localName, permissions, + modificationTime, atime, (BlockInfoContiguous[]) blocks, + replication, blockSize); if (underConstruction) { file.toUnderConstruction(clientName, clientMachine); } @@ -1315,7 +1276,6 @@ void save(File newFile, FSImageCompression compression) throws IOException { out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV2()); out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampAtblockIdSwitch()); out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedContiguousBlockId()); - out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedStripedBlockId()); out.writeLong(context.getTxId()); out.writeLong(sourceNamesystem.dir.getLastInodeId()); 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 47447166913ad..af3f8139d13d4 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 @@ -32,12 +32,9 @@ 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.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap; @@ -50,7 +47,6 @@ import org.apache.hadoop.io.ShortWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.xml.sax.ContentHandler; import org.xml.sax.SAXException; @@ -128,45 +124,22 @@ static INodeFile readINodeUnderConstruction( short blockReplication = in.readShort(); long modificationTime = in.readLong(); long preferredBlockSize = in.readLong(); - final boolean isStriped = NameNodeLayoutVersion.supports( - NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion) - && (in.readBoolean()); - - // TODO: ECSchema can be restored from persisted file (HDFS-7859). - final ECSchema schema = isStriped ? - ErasureCodingSchemaManager.getSystemDefaultSchema() : null; int numBlocks = in.readInt(); - final BlockInfoContiguous[] blocksContiguous; - BlockInfoStriped[] blocksStriped = null; - if (isStriped) { - blocksContiguous = new BlockInfoContiguous[0]; - blocksStriped = new BlockInfoStriped[numBlocks]; - int i = 0; - for (; i < numBlocks - 1; i++) { - blocksStriped[i] = new BlockInfoStriped(new Block(), schema); - blocksStriped[i].readFields(in); - } - if (numBlocks > 0) { - blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(), - schema, BlockUCState.UNDER_CONSTRUCTION, null); - blocksStriped[i].readFields(in); - } - } else { - blocksContiguous = new BlockInfoContiguous[numBlocks]; - Block blk = new Block(); - int i = 0; - for (; i < numBlocks-1; i++) { - blk.readFields(in); - blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication); - } - // last block is UNDER_CONSTRUCTION - if(numBlocks > 0) { - blk.readFields(in); - blocksContiguous[i] = new BlockInfoContiguousUnderConstruction( - blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); - } + final BlockInfoContiguous[] blocksContiguous = + new BlockInfoContiguous[numBlocks]; + Block blk = new Block(); + int i = 0; + for (; i < numBlocks - 1; i++) { + blk.readFields(in); + blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication); + } + // last block is UNDER_CONSTRUCTION + if(numBlocks > 0) { + blk.readFields(in); + blocksContiguous[i] = new BlockInfoContiguousUnderConstruction( + blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); } PermissionStatus perm = PermissionStatus.read(in); @@ -180,19 +153,8 @@ static INodeFile readINodeUnderConstruction( // Images in the pre-protobuf format will not have the lazyPersist flag, // so it is safe to pass false always. - INodeFile file; - if (isStriped) { - file = new INodeFile(inodeId, name, perm, modificationTime, - modificationTime, blocksContiguous, (short) 0, preferredBlockSize); - file.addStripedBlocksFeature(); - for (int i = 0; i < numBlocks; i++) { - file.getStripedBlocksFeature().addBlock(blocksStriped[i]); - } - } else { - file = new INodeFile(inodeId, name, perm, modificationTime, - modificationTime, blocksContiguous, blockReplication, - preferredBlockSize); - } + INodeFile file = new INodeFile(inodeId, name, perm, modificationTime, + modificationTime, blocksContiguous, blockReplication, preferredBlockSize); file.toUnderConstruction(clientName, clientMachine); return file; } @@ -207,8 +169,7 @@ static void writeINodeUnderConstruction(DataOutputStream out, INodeFile cons, out.writeShort(cons.getFileReplication()); out.writeLong(cons.getModificationTime()); out.writeLong(cons.getPreferredBlockSize()); - // whether the file has striped blocks - out.writeBoolean(cons.isStriped()); + writeBlocks(cons.getBlocks(), out); cons.getPermissionStatus().write(out); @@ -233,8 +194,7 @@ public static void writeINodeFile(INodeFile file, DataOutput out, out.writeLong(file.getModificationTime()); out.writeLong(file.getAccessTime()); out.writeLong(file.getPreferredBlockSize()); - // whether the file has striped blocks - out.writeBoolean(file.isStriped()); + writeBlocks(file.getBlocks(), out); SnapshotFSImageFormat.saveFileDiffList(file, out); @@ -347,7 +307,7 @@ private static void writeINodeReference(INodeReference ref, DataOutput out, if (!isWithName) { Preconditions.checkState(ref instanceof INodeReference.DstReference); // dst snapshot id - out.writeInt(((INodeReference.DstReference) ref).getDstSnapshotId()); + out.writeInt(ref.getDstSnapshotId()); } else { out.writeInt(((INodeReference.WithName) ref).getLastSnapshotId()); } 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 8fd0753448e8d..ef61d68212f2a 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 @@ -203,27 +203,11 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con fileByLoaded.getPermissionStatus().getPermission()); assertEquals(mtime, fileByLoaded.getModificationTime()); assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime()); - assertEquals(0, fileByLoaded.getContiguousBlocks().length); - assertEquals(0, fileByLoaded.getFileReplication()); + // TODO for striped blocks, we currently save and load them as contiguous + // blocks to/from legacy fsimage + assertEquals(3, fileByLoaded.getContiguousBlocks().length); assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize()); - //check the BlockInfoStriped - BlockInfoStriped[] stripedBlksByLoaded = - fileByLoaded.getStripedBlocksFeature().getBlocks(); - assertEquals(3, stripedBlksByLoaded.length); - for (int i = 0; i < 3; i++) { - assertEquals(stripedBlks[i].getBlockId(), - stripedBlksByLoaded[i].getBlockId()); - assertEquals(stripedBlks[i].getNumBytes(), - stripedBlksByLoaded[i].getNumBytes()); - assertEquals(stripedBlks[i].getGenerationStamp(), - stripedBlksByLoaded[i].getGenerationStamp()); - assertEquals(stripedBlks[i].getDataBlockNum(), - stripedBlksByLoaded[i].getDataBlockNum()); - assertEquals(stripedBlks[i].getParityBlockNum(), - stripedBlksByLoaded[i].getParityBlockNum()); - } - if (isUC) { assertEquals(client, fileByLoaded.getFileUnderConstructionFeature().getClientName()); From c12a974ccf5f52f63e4f825d8b4d2385953cd119 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 17 Jun 2015 11:35:31 -0700 Subject: [PATCH 159/212] HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and StripedBlocksFeature. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../blockmanagement/BlockInfoStriped.java | 12 +++-- .../BlockInfoStripedUnderConstruction.java | 9 ++-- .../server/namenode/FSDirWriteFileOp.java | 8 ++-- .../hdfs/server/namenode/FSEditLogLoader.java | 45 +++++++++++-------- .../server/namenode/FSImageFormatPBINode.java | 20 ++++++--- .../hdfs/server/namenode/FSNamesystem.java | 7 +-- .../hadoop-hdfs/src/main/proto/fsimage.proto | 1 + .../blockmanagement/TestBlockInfoStriped.java | 6 ++- .../server/namenode/TestFSEditLogLoader.java | 9 +++- .../hdfs/server/namenode/TestFSImage.java | 3 +- .../server/namenode/TestStripedINodeFile.java | 27 +++++------ 12 files changed, 91 insertions(+), 59 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 1ae3e9b3fad87..3c840cc7684d4 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -302,3 +302,6 @@ HDFS-8559. Erasure Coding: fix non-protobuf fsimage for striped blocks. (Jing Zhao via yliu) + + HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and + StripedBlocksFeature. (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 3898a495ada83..4cede91281990 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -39,6 +39,7 @@ */ public class BlockInfoStriped extends BlockInfo { private final ECSchema schema; + private final int cellSize; /** * Always the same size with triplets. Record the block index for each triplet * TODO: actually this is only necessary for over-replicated block. Thus can @@ -46,15 +47,16 @@ public class BlockInfoStriped extends BlockInfo { */ private byte[] indices; - public BlockInfoStriped(Block blk, ECSchema schema) { + public BlockInfoStriped(Block blk, ECSchema schema, int cellSize) { super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits())); indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()]; initIndices(); this.schema = schema; + this.cellSize = cellSize; } BlockInfoStriped(BlockInfoStriped b) { - this(b, b.getSchema()); + this(b, b.getSchema(), b.getCellSize()); this.setBlockCollection(b.getBlockCollection()); } @@ -75,6 +77,10 @@ public ECSchema getSchema() { return schema; } + public int getCellSize() { + return cellSize; + } + private void initIndices() { for (int i = 0; i < indices.length; i++) { indices[i] = -1; @@ -236,7 +242,7 @@ public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( BlockUCState s, DatanodeStorageInfo[] targets) { final BlockInfoStripedUnderConstruction ucBlock; if(isComplete()) { - ucBlock = new BlockInfoStripedUnderConstruction(this, schema, + ucBlock = new BlockInfoStripedUnderConstruction(this, schema, cellSize, s, targets); ucBlock.setBlockCollection(getBlockCollection()); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java index 76d79201b511b..5f78096d719ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java @@ -57,16 +57,17 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped /** * Constructor with null storage targets. */ - public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema) { - this(blk, schema, UNDER_CONSTRUCTION, null); + public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, + int cellSize) { + this(blk, schema, cellSize, UNDER_CONSTRUCTION, null); } /** * Create a striped block that is currently being constructed. */ public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, - BlockUCState state, DatanodeStorageInfo[] targets) { - super(blk, schema); + int cellSize, BlockUCState state, DatanodeStorageInfo[] targets) { + super(blk, schema, cellSize); assert getBlockUCState() != COMPLETE : "BlockInfoStripedUnderConstruction cannot be in COMPLETE state"; this.blockUCState = state; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index a3daabcc59da7..e92388bcd6ca5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -531,8 +532,9 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, // associate new last block for the file final BlockInfo blockInfo; if (isStriped) { - ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsd.getFSNamesystem(), inodesInPath); + ECSchema ecSchema = ecZone.getSchema(); short numDataUnits = (short) ecSchema.getNumDataUnits(); short numParityUnits = (short) ecSchema.getNumParityUnits(); short numLocations = (short) (numDataUnits + numParityUnits); @@ -541,8 +543,8 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), numLocations, true); blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema, - HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, - targets); + ecZone.getCellSize(), + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } else { // check quota limits and updated space consumed fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), 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 62a278cd5a39e..76ae561e4a43f 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 @@ -36,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; @@ -417,9 +418,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsDir.getFSNamesystem(), iip); - updateBlocks(fsDir, addCloseOp, iip, newFile, ecSchema); + updateBlocks(fsDir, addCloseOp, iip, newFile, ecZone); break; } case OP_CLOSE: { @@ -439,9 +440,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsDir.getFSNamesystem(), iip); - updateBlocks(fsDir, addCloseOp, iip, file, ecSchema); + updateBlocks(fsDir, addCloseOp, iip, file, ecZone); // Now close the file if (!file.isUnderConstruction() && @@ -499,9 +500,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures - ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsDir.getFSNamesystem(), iip); - updateBlocks(fsDir, updateOp, iip, oldFile, ecSchema); + updateBlocks(fsDir, updateOp, iip, oldFile, ecZone); if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); @@ -518,9 +519,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // add the new block to the INodeFile - ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( + ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsDir.getFSNamesystem(), iip); - addNewBlock(addBlockOp, oldFile, ecSchema); + addNewBlock(addBlockOp, oldFile, ecZone); break; } case OP_SET_REPLICATION: { @@ -961,8 +962,8 @@ private static String formatEditLogReplayError(EditLogInputStream in, /** * Add a new block into the given INodeFile */ - private void addNewBlock(AddBlockOp op, INodeFile file, ECSchema ecSchema) - throws IOException { + private void addNewBlock(AddBlockOp op, INodeFile file, + ErasureCodingZone ecZone) throws IOException { BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); Block newBlock= op.getLastBlock(); @@ -989,9 +990,10 @@ private void addNewBlock(AddBlockOp op, INodeFile file, ECSchema ecSchema) } // add the new block final BlockInfo newBlockInfo; - boolean isStriped = ecSchema != null; + boolean isStriped = ecZone != null; if (isStriped) { - newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, ecSchema); + newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, + ecZone.getSchema(), ecZone.getCellSize()); } else { newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); @@ -1006,7 +1008,8 @@ private void addNewBlock(AddBlockOp op, INodeFile file, ECSchema ecSchema) * @throws IOException */ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, - INodesInPath iip, INodeFile file, ECSchema ecSchema) throws IOException { + INodesInPath iip, INodeFile file, ErasureCodingZone ecZone) + throws IOException { // Update its block list BlockInfo[] oldBlocks = file.getBlocks(); Block[] newBlocks = op.getBlocks(); @@ -1065,7 +1068,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, throw new IOException("Trying to delete non-existant block " + oldBlock); } } else if (newBlocks.length > oldBlocks.length) { - final boolean isStriped = ecSchema != null; + final boolean isStriped = ecZone != null; // We're adding blocks for (int i = oldBlocks.length; i < newBlocks.length; i++) { Block newBlock = newBlocks[i]; @@ -1075,7 +1078,8 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // what about an old-version fsync() where fsync isn't called // until several blocks in? if (isStriped) { - newBI = new BlockInfoStripedUnderConstruction(newBlock, ecSchema); + newBI = new BlockInfoStripedUnderConstruction(newBlock, + ecZone.getSchema(), ecZone.getCellSize()); } else { newBI = new BlockInfoContiguousUnderConstruction(newBlock, file.getPreferredBlockReplication()); @@ -1086,9 +1090,14 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // versions of Hadoop. Current versions always log // OP_ADD operations as each block is allocated. // TODO: ECSchema can be restored from persisted file (HDFS-7859). - newBI = isStriped ? new BlockInfoStriped(newBlock, - ErasureCodingSchemaManager.getSystemDefaultSchema()) : - new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); + if (isStriped) { + newBI = new BlockInfoStriped(newBlock, + ErasureCodingSchemaManager.getSystemDefaultSchema(), + ecZone.getCellSize()); + } else { + newBI = new BlockInfoContiguous(newBlock, + file.getPreferredBlockReplication()); + } } fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file); file.addBlock(newBI); 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 92e705f30a506..1f8453911f4b2 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 @@ -360,8 +360,13 @@ private INodeFile loadINodeFile(INodeSection.INode n) { // TODO: HDFS-7859 ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); stripeFeature = file.addStripedBlocksFeature(); - for (BlockProto b : bp) { - stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b), schema)); + if (bp.size() > 0) { + // if a striped file has block, the cellSize must exist in proto + final int cellSize = f.getStripedBlocks().getCellSize(); + for (BlockProto b : bp) { + stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b), + schema, cellSize)); + } } } @@ -376,7 +381,7 @@ private INodeFile loadINodeFile(INodeSection.INode n) { if (stripeFeature != null) { BlockInfoStriped striped = (BlockInfoStriped) lastBlk; ucBlk = new BlockInfoStripedUnderConstruction(striped, - striped.getSchema()); + striped.getSchema(), striped.getCellSize()); } else { ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication); @@ -659,14 +664,17 @@ private void save(OutputStream out, INodeFile n) throws IOException { FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature(); if (sb != null) { + StripedBlocksFeature.Builder builder = + StripedBlocksFeature.newBuilder(); BlockInfoStriped[] sblocks = sb.getBlocks(); - if (sblocks != null) { + if (sblocks != null && sblocks.length > 0) { + final int cellSize = sblocks[0].getCellSize(); for (BlockInfoStriped sblk : sblocks) { + assert cellSize == sblk.getCellSize(); b.addBlocks(PBHelper.convert(sblk)); } + builder.setCellSize(cellSize); } - StripedBlocksFeature.Builder builder = - StripedBlocksFeature.newBuilder(); b.setStripedBlocks(builder.build()); } 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 655ead891895f..1dd0a432d6aaa 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 @@ -3675,8 +3675,9 @@ void commitOrCompleteLastBlock( final long diff; final short replicationFactor; if (fileINode.isStriped()) { - final ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema( - this, iip); + final ErasureCodingZone ecZone = FSDirErasureCodingOp + .getErasureCodingZone(this, iip); + final ECSchema ecSchema = ecZone.getSchema(); final short numDataUnits = (short) ecSchema.getNumDataUnits(); final short numParityUnits = (short) ecSchema.getNumParityUnits(); @@ -3685,7 +3686,7 @@ void commitOrCompleteLastBlock( fileINode.getPreferredBlockSize() * numBlocks; final BlockInfoStriped striped = new BlockInfoStriped(commitBlock, - ecSchema); + ecSchema, ecZone.getCellSize()); final long actualBlockGroupSize = striped.spaceConsumed(); diff = fullBlockGroupSize - actualBlockGroupSize; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto index 4d0e2ddea89ff..5bb1c3242d2b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto @@ -94,6 +94,7 @@ message INodeSection { message StripedBlocksFeature { // store striped blocks related information + optional uint32 cellSize = 1; } message AclFeatureProto { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java index 2d6b5092d7a34..6788770f70178 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -19,6 +19,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -46,8 +47,9 @@ public class TestBlockInfoStriped { private static final Block baseBlock = new Block(BASE_ID); private static final ECSchema testSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); + private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private final BlockInfoStriped info = new BlockInfoStriped(baseBlock, - testSchema); + testSchema, cellSize); private Block[] createReportedBlocks(int num) { Block[] blocks = new Block[num]; @@ -235,7 +237,7 @@ public void testWrite() { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutput out = new DataOutputStream(byteStream); BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes, - generationStamp), testSchema); + generationStamp), testSchema, cellSize); try { blk.write(out); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index abb9bf5aea796..79cb4957c82b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -448,6 +449,7 @@ public void testAddNewStripedBlock() throws IOException{ long timestamp = 1426222918; short blockNum = HdfsConstants.NUM_DATA_BLOCKS; short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; + int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); @@ -463,7 +465,7 @@ public void testAddNewStripedBlock() throws IOException{ // Add a striped block to the file BlockInfoStriped stripedBlk = new BlockInfoStriped( - new Block(blkId, blkNumBytes, timestamp), testSchema); + new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); file.getStripedBlocksFeature().addBlock(stripedBlk); @@ -488,6 +490,7 @@ public void testAddNewStripedBlock() throws IOException{ assertEquals(timestamp, blks[0].getGenerationStamp()); assertEquals(blockNum, blks[0].getDataBlockNum()); assertEquals(parityNum, blks[0].getParityBlockNum()); + assertEquals(cellSize, blks[0].getCellSize()); cluster.shutdown(); cluster = null; @@ -520,6 +523,7 @@ public void testUpdateStripedBlocks() throws IOException{ long timestamp = 1426222918; short blockNum = HdfsConstants.NUM_DATA_BLOCKS; short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; + int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); @@ -529,7 +533,7 @@ public void testUpdateStripedBlocks() throws IOException{ Path p = new Path(testFilePath); DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); BlockInfoStriped stripedBlk = new BlockInfoStriped( - new Block(blkId, blkNumBytes, timestamp), testSchema); + new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); file.getStripedBlocksFeature().addBlock(stripedBlk); @@ -567,6 +571,7 @@ public void testUpdateStripedBlocks() throws IOException{ assertEquals(newTimestamp, blks[0].getGenerationStamp()); assertEquals(blockNum, blks[0].getDataBlockNum()); assertEquals(parityNum, blks[0].getParityBlockNum()); + assertEquals(cellSize, blks[0].getCellSize()); cluster.shutdown(); cluster = null; 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 ef61d68212f2a..c55b9ba80fb52 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 @@ -70,6 +70,7 @@ public class TestFSImage { "image-with-zero-block-size.tar.gz"; private static final ECSchema testSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); + private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @Test public void testPersist() throws IOException { @@ -162,7 +163,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con for (int i = 0; i < stripedBlks.length; i++) { stripedBlks[i] = new BlockInfoStriped( new Block(stripedBlkId + i, preferredBlockSize, timestamp), - testSchema); + testSchema, cellSize); file.getStripedBlocksFeature().addBlock(stripedBlks[i]); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 9e4da843bd90f..c0022bd519f44 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -53,6 +53,7 @@ public class TestStripedINodeFile { private static final ECSchema testSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); + private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; private static INodeFile createStripedINodeFile() { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, @@ -69,22 +70,20 @@ public void testBlockStripedFeature() @Test public void testBlockStripedTotalBlockCount() { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema); + = new BlockInfoStriped(blk, testSchema, cellSize); assertEquals(9, blockInfoStriped.getTotalBlockNum()); } @Test public void testBlockStripedLength() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema); + = new BlockInfoStriped(blk, testSchema, cellSize); inf.addBlock(blockInfoStriped); assertEquals(1, inf.getBlocks().length); } @@ -92,12 +91,11 @@ public void testBlockStripedLength() @Test public void testBlockStripedConsumedSpace() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema); + = new BlockInfoStriped(blk, testSchema, cellSize); blockInfoStriped.setNumBytes(1); inf.addBlock(blockInfoStriped); // 0. Calculate the total bytes per stripes @@ -119,16 +117,15 @@ public void testBlockStripedConsumedSpace() @Test public void testMultipleBlockStripedConsumedSpace() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk1 = new Block(1); BlockInfoStriped blockInfoStriped1 - = new BlockInfoStriped(blk1, testSchema); + = new BlockInfoStriped(blk1, testSchema, cellSize); blockInfoStriped1.setNumBytes(1); Block blk2 = new Block(2); BlockInfoStriped blockInfoStriped2 - = new BlockInfoStriped(blk2, testSchema); + = new BlockInfoStriped(blk2, testSchema, cellSize); blockInfoStriped2.setNumBytes(1); inf.addBlock(blockInfoStriped1); inf.addBlock(blockInfoStriped2); @@ -140,12 +137,11 @@ public void testMultipleBlockStripedConsumedSpace() @Test public void testBlockStripedFileSize() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema); + = new BlockInfoStriped(blk, testSchema, cellSize); blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); // Compute file size should return actual data @@ -157,12 +153,11 @@ public void testBlockStripedFileSize() @Test public void testBlockStripedUCFileSize() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStripedUnderConstruction bInfoStripedUC - = new BlockInfoStripedUnderConstruction(blk, testSchema); + = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); bInfoStripedUC.setNumBytes(100); inf.addBlock(bInfoStripedUC); assertEquals(100, inf.computeFileSize()); @@ -172,12 +167,11 @@ public void testBlockStripedUCFileSize() @Test public void testBlockStripedComputeQuotaUsage() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema); + = new BlockInfoStriped(blk, testSchema, cellSize); blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); @@ -195,12 +189,11 @@ public void testBlockStripedComputeQuotaUsage() @Test public void testBlockStripedUCComputeQuotaUsage() throws IOException, InterruptedException { - ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); INodeFile inf = createStripedINodeFile(); inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStripedUnderConstruction bInfoStripedUC - = new BlockInfoStripedUnderConstruction(blk, testSchema); + = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); bInfoStripedUC.setNumBytes(100); inf.addBlock(bInfoStripedUC); From 62c5a879ef2129b6d11752ccea099d461d29b4b7 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Thu, 18 Jun 2015 14:15:01 +0530 Subject: [PATCH 160/212] HDFS-8466. Refactor BlockInfoContiguous and fix NPE in TestBlockInfo#testCopyConstructor() (Contributed by Vinayakumar B) --- .../blockmanagement/BlockInfoContiguous.java | 57 ++----------------- 1 file changed, 4 insertions(+), 53 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index 416091fba5b27..5199101267d47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -37,65 +37,16 @@ public BlockInfoContiguous(Block blk, short size) { } /** - * Copy construction. - * This is used to convert BlockReplicationInfoUnderConstruction + * Copy construction. This is used to convert + * BlockReplicationInfoUnderConstruction + * * @param from BlockReplicationInfo to copy from. */ protected BlockInfoContiguous(BlockInfoContiguous from) { - this(from, from.getBlockCollection().getPreferredBlockReplication()); - this.triplets = new Object[from.triplets.length]; + this(from, (short) (from.triplets.length / 3)); this.setBlockCollection(from.getBlockCollection()); } - 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 (DatanodeStorageInfo)triplets[index*3]; - } - - /** - * Return the previous block on the block list for the datanode at - * position index. Set the previous block on the list to "to". - * - * @param index - the datanode index - * @param to - block to be set to previous on the list of blocks - * @return current previous block on the list of blocks - */ - private BlockInfoContiguous setPrevious(int index, BlockInfoContiguous to) { - assert this.triplets != null : "BlockInfo is not initialized"; - assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; - BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1]; - triplets[index*3+1] = to; - return info; - } - - /** - * Return the next block on the block list for the datanode at - * position index. Set the next block on the list to "to". - * - * @param index - the datanode index - * @param to - block to be set to next on the list of blocks - * * @return current next block on the list of blocks - */ - private BlockInfoContiguous setNext(int index, BlockInfoContiguous to) { - assert this.triplets != null : "BlockInfo is not initialized"; - assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; - BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2]; - triplets[index*3+2] = to; - return info; - } - - public int getCapacity() { - assert this.triplets != null : "BlockInfo is not initialized"; - assert triplets.length % 3 == 0 : "Malformed BlockInfo"; - return triplets.length / 3; - } - /** * Ensure that there is enough space to include num more triplets. * @return first free triplet index. From 05c696882ee0844052247cab5451e4bf7bb12a0f Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Thu, 18 Jun 2015 14:19:49 +0530 Subject: [PATCH 161/212] HDFS-8466. Refactor BlockInfoContiguous and fix NPE in TestBlockInfo#testCopyConstructor() (Contributed by Vinayakumar B) Missed CHANGES.txt entry --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 3c840cc7684d4..f41d30a678ed4 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -305,3 +305,6 @@ HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and StripedBlocksFeature. (Walter Su via jing9) + + HDFS-8466. Refactor BlockInfoContiguous and fix NPE in + TestBlockInfo#testCopyConstructor() (vinayakumarb) \ No newline at end of file From 3682e01984b6d93b35376532da8a8823d69239df Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 19 Jun 2015 10:23:45 -0700 Subject: [PATCH 162/212] HDFS-8254. Avoid assigning a leading streamer in StripedDataStreamer to tolerate datanode failure. Contributed by Tsz Wo Nicholas Sze. --- .../hdfs/client/HdfsClientConfigKeys.java | 10 - .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 192 +++++++++------- .../org/apache/hadoop/hdfs/DataStreamer.java | 49 +++-- .../hadoop/hdfs/StripedDataStreamer.java | 206 ++++++++++++------ .../hdfs/client/impl/DfsClientConf.java | 23 -- .../hadoop/hdfs/util/StripedBlockUtil.java | 24 +- ...TestDFSStripedOutputStreamWithFailure.java | 7 + 8 files changed, 307 insertions(+), 209 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 9373e98bf42c1..6006d71faf6a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -189,16 +189,6 @@ interface StripedRead { int THREADPOOL_SIZE_DEFAULT = 18; } - /** dfs.client.write.striped configuration properties */ - interface StripedWrite { - String PREFIX = Write.PREFIX + "striped."; - - String MAX_SECONDS_GET_STRIPED_BLOCK_KEY = PREFIX + "max-seconds-get-striped-block"; - int MAX_SECONDS_GET_STRIPED_BLOCK_DEFAULT = 90; - String MAX_SECONDS_GET_ENDED_BLOCK_KEY = PREFIX + "max-seconds-get-ended-block"; - int MAX_SECONDS_GET_ENDED_BLOCK_DEFAULT = 60; - } - /** dfs.http.client configuration properties */ interface HttpClient { String PREFIX = "dfs.http.client."; diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index f41d30a678ed4..a710c2efa81f0 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -307,4 +307,7 @@ StripedBlocksFeature. (Walter Su via jing9) HDFS-8466. Refactor BlockInfoContiguous and fix NPE in - TestBlockInfo#testCopyConstructor() (vinayakumarb) \ No newline at end of file + TestBlockInfo#testCopyConstructor() (vinayakumarb) + + HDFS-8254. Avoid assigning a leading streamer in StripedDataStreamer to + tolerate datanode failure. (Tsz Wo Nicholas Sze via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index bdd3352179346..1068b3752ca25 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; @@ -40,7 +39,6 @@ import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; @@ -51,27 +49,33 @@ import com.google.common.base.Preconditions; -/**************************************************************** - * The DFSStripedOutputStream class supports writing files in striped - * layout. Each stripe contains a sequence of cells and multiple - * {@link StripedDataStreamer}s in DFSStripedOutputStream are responsible - * for writing the cells to different datanodes. - * - ****************************************************************/ - +/** + * This class supports writing files in striped layout and erasure coded format. + * Each stripe contains a sequence of cells. + */ @InterfaceAudience.Private public class DFSStripedOutputStream extends DFSOutputStream { static class MultipleBlockingQueue { - private final int pullTimeout; private final List> queues; - MultipleBlockingQueue(int numQueue, int queueSize, int pullTimeout) { + MultipleBlockingQueue(int numQueue, int queueSize) { queues = new ArrayList<>(numQueue); for (int i = 0; i < numQueue; i++) { queues.add(new LinkedBlockingQueue(queueSize)); } + } + + boolean isEmpty() { + for(int i = 0; i < queues.size(); i++) { + if (!queues.get(i).isEmpty()) { + return false; + } + } + return true; + } - this.pullTimeout = pullTimeout; + int numQueues() { + return queues.size(); } void offer(int i, T object) { @@ -80,49 +84,71 @@ void offer(int i, T object) { + " to queue, i=" + i); } - T poll(int i) throws InterruptedIOException { + T take(int i) throws InterruptedIOException { try { - return queues.get(i).poll(pullTimeout, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw DFSUtil.toInterruptedIOException("poll interrupted, i=" + i, e); + return queues.get(i).take(); + } catch(InterruptedException ie) { + throw DFSUtil.toInterruptedIOException("take interrupted, i=" + i, ie); } } + T poll(int i) { + return queues.get(i).poll(); + } + T peek(int i) { return queues.get(i).peek(); } } /** Coordinate the communication between the streamers. */ - static class Coordinator { - private final MultipleBlockingQueue stripedBlocks; + class Coordinator { + private final MultipleBlockingQueue followingBlocks; private final MultipleBlockingQueue endBlocks; + + private final MultipleBlockingQueue newBlocks; private final MultipleBlockingQueue updateBlocks; Coordinator(final DfsClientConf conf, final int numDataBlocks, final int numAllBlocks) { - stripedBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1, - conf.getStripedWriteMaxSecondsGetStripedBlock()); - endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1, - conf.getStripedWriteMaxSecondsGetEndedBlock()); - updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1, - conf.getStripedWriteMaxSecondsGetStripedBlock()); + followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); + endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1); + + newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); + updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); } - void putEndBlock(int i, ExtendedBlock block) { - endBlocks.offer(i, block); + MultipleBlockingQueue getFollowingBlocks() { + return followingBlocks; + } + + MultipleBlockingQueue getNewBlocks() { + return newBlocks; } - ExtendedBlock getEndBlock(int i) throws InterruptedIOException { - return endBlocks.poll(i); + MultipleBlockingQueue getUpdateBlocks() { + return updateBlocks; } - void putUpdateBlock(int i, ExtendedBlock block) { - updateBlocks.offer(i, block); + StripedDataStreamer getStripedDataStreamer(int i) { + return DFSStripedOutputStream.this.getStripedDataStreamer(i); } - ExtendedBlock getUpdateBlock(int i) throws InterruptedIOException { - return updateBlocks.poll(i); + void offerEndBlock(int i, ExtendedBlock block) { + endBlocks.offer(i, block); + } + + ExtendedBlock takeEndBlock(int i) throws InterruptedIOException { + return endBlocks.take(i); + } + + boolean hasAllEndBlocks() { + for(int i = 0; i < endBlocks.numQueues(); i++) { + if (endBlocks.peek(i) == null) { + return false; + } + } + return true; } void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) { @@ -130,24 +156,35 @@ void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) { if (b == null) { // streamer just has failed, put end block and continue b = block; - putEndBlock(i, b); + offerEndBlock(i, b); } b.setNumBytes(newBytes); } - void putStripedBlock(int i, LocatedBlock block) throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("putStripedBlock " + block + ", i=" + i); + /** @return a block representing the entire block group. */ + ExtendedBlock getBlockGroup() { + final StripedDataStreamer s0 = getStripedDataStreamer(0); + final ExtendedBlock b0 = s0.getBlock(); + if (b0 == null) { + return null; } - stripedBlocks.offer(i, block); - } - LocatedBlock getStripedBlock(int i) throws IOException { - final LocatedBlock lb = stripedBlocks.poll(i); - if (lb == null) { - throw new IOException("Failed: i=" + i); + final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0; + final ExtendedBlock block = new ExtendedBlock(b0); + long numBytes = b0.getNumBytes(); + for (int i = 1; i < numDataBlocks; i++) { + final StripedDataStreamer si = getStripedDataStreamer(i); + final ExtendedBlock bi = si.getBlock(); + if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) { + block.setGenerationStamp(bi.getGenerationStamp()); + } + numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock(); } - return lb; + block.setNumBytes(numBytes); + if (LOG.isDebugEnabled()) { + LOG.debug("getBlockGroup: " + block + ", numBytes=" + block.getNumBytes()); + } + return block; } } @@ -223,13 +260,9 @@ private void flipDataBuffers() { private final int numAllBlocks; private final int numDataBlocks; - private StripedDataStreamer getLeadingStreamer() { - return streamers.get(0); - } - @Override ExtendedBlock getBlock() { - return getLeadingStreamer().getBlock(); + return coordinator.getBlockGroup(); } /** Construct a new output stream for creating a file. */ @@ -308,7 +341,9 @@ private void checkStreamers() throws IOException { int count = 0; for(StripedDataStreamer s : streamers) { if (!s.isFailed()) { - s.getErrorState().initExtenalError(); + if (s.getBlock() != null) { + s.getErrorState().initExternalError(); + } count++; } } @@ -325,7 +360,7 @@ private void checkStreamers() throws IOException { private void handleStreamerFailure(String err, Exception e) throws IOException { LOG.warn("Failed: " + err + ", " + this, e); - getCurrentStreamer().setIsFailed(true); + getCurrentStreamer().setFailed(true); checkStreamers(); currentPacket = null; } @@ -443,10 +478,17 @@ synchronized void abort() throws IOException { dfsClient.endFileLease(fileId); } - //TODO: Handle slow writers (HDFS-7786) - //Cuurently only check if the leading streamer is terminated + @Override boolean isClosed() { - return closed || getLeadingStreamer().streamerClosed(); + if (closed) { + return true; + } + for(StripedDataStreamer s : streamers) { + if (!s.streamerClosed()) { + return false; + } + } + return true; } @Override @@ -560,7 +602,19 @@ void setClosed() { @Override protected synchronized void closeImpl() throws IOException { if (isClosed()) { - getLeadingStreamer().getLastException().check(true); + final MultipleIOException.Builder b = new MultipleIOException.Builder(); + for(int i = 0; i < streamers.size(); i++) { + final StripedDataStreamer si = getStripedDataStreamer(i); + try { + si.getLastException().check(true); + } catch (IOException e) { + b.add(e); + } + } + final IOException ioe = b.build(); + if (ioe != null) { + throw ioe; + } return; } @@ -594,7 +648,7 @@ protected synchronized void closeImpl() throws IOException { } closeThreads(false); - final ExtendedBlock lastBlock = getCommittedBlock(); + final ExtendedBlock lastBlock = coordinator.getBlockGroup(); TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER); try { completeFile(lastBlock); @@ -607,30 +661,4 @@ protected synchronized void closeImpl() throws IOException { setClosed(); } } - - /** - * Generate the block which is reported and will be committed in NameNode. - * Need to go through all the streamers writing data blocks and add their - * bytesCurBlock together. Note that at this time all streamers have been - * closed. Also this calculation can cover streamers with writing failures. - * - * @return An ExtendedBlock with size of the whole block group. - */ - ExtendedBlock getCommittedBlock() throws IOException { - ExtendedBlock b = getLeadingStreamer().getBlock(); - if (b == null) { - return null; - } - final ExtendedBlock block = new ExtendedBlock(b); - final boolean atBlockGroupBoundary = - getLeadingStreamer().getBytesCurBlock() == 0 && - getLeadingStreamer().getBlock() != null && - getLeadingStreamer().getBlock().getNumBytes() > 0; - for (int i = 1; i < numDataBlocks; i++) { - block.setNumBytes(block.getNumBytes() + - (atBlockGroupBoundary ? streamers.get(i).getBlock().getNumBytes() : - streamers.get(i).getBytesCurBlock())); - } - return block; - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index 1344d54d61a0c..c78199ed3067d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -209,7 +209,7 @@ synchronized void throwException4Close() throws IOException { static class ErrorState { private boolean error = false; - private boolean extenalError = false; + private boolean externalError = false; private int badNodeIndex = -1; private int restartingNodeIndex = -1; private long restartingNodeDeadline = 0; @@ -221,7 +221,7 @@ static class ErrorState { synchronized void reset() { error = false; - extenalError = false; + externalError = false; badNodeIndex = -1; restartingNodeIndex = -1; restartingNodeDeadline = 0; @@ -231,17 +231,21 @@ synchronized boolean hasError() { return error; } + synchronized boolean hasExternalErrorOnly() { + return error && externalError && !isNodeMarked(); + } + synchronized boolean hasDatanodeError() { - return error && (isNodeMarked() || extenalError); + return error && (isNodeMarked() || externalError); } synchronized void setError(boolean err) { this.error = err; } - synchronized void initExtenalError() { + synchronized void initExternalError() { setError(true); - this.extenalError = true; + this.externalError = true; } @@ -405,11 +409,13 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { private final LoadingCache excludedNodes; private final String[] favoredNodes; - private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, + private DataStreamer(HdfsFileStatus stat, ExtendedBlock block, + DFSClient dfsClient, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage, boolean isAppend, String[] favoredNodes) { + this.block = block; this.dfsClient = dfsClient; this.src = src; this.progress = progress; @@ -434,9 +440,8 @@ private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage, String[] favoredNodes) { - this(stat, dfsClient, src, progress, checksum, cachingStrategy, + this(stat, block, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManage, false, favoredNodes); - this.block = block; stage = BlockConstructionStage.PIPELINE_SETUP_CREATE; } @@ -450,10 +455,9 @@ private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage) throws IOException { - this(stat, dfsClient, src, progress, checksum, cachingStrategy, + this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy, byteArrayManage, true, null); stage = BlockConstructionStage.PIPELINE_SETUP_APPEND; - block = lastBlock.getBlock(); bytesSent = block.getNumBytes(); accessToken = lastBlock.getBlockToken(); } @@ -1074,6 +1078,10 @@ private boolean processDatanodeError() throws IOException { if (!errorState.hasDatanodeError()) { return false; } + if (errorState.hasExternalErrorOnly() && block == null) { + // block is not yet initialized, handle external error later. + return false; + } if (response != null) { LOG.info("Error Recovery for " + block + " waiting for responder to exit. "); @@ -1402,15 +1410,28 @@ private void failPacket4Testing() { } LocatedBlock updateBlockForPipeline() throws IOException { + return callUpdateBlockForPipeline(block); + } + + LocatedBlock callUpdateBlockForPipeline(ExtendedBlock newBlock) throws IOException { return dfsClient.namenode.updateBlockForPipeline( - block, dfsClient.clientName); + newBlock, dfsClient.clientName); + } + + static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) { + return new ExtendedBlock(b.getBlockPoolId(), b.getBlockId(), + b.getNumBytes(), newGS); } /** update pipeline at the namenode */ ExtendedBlock updatePipeline(long newGS) throws IOException { - final ExtendedBlock newBlock = new ExtendedBlock( - block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS); - dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, + final ExtendedBlock newBlock = newBlock(block, newGS); + return callUpdatePipeline(block, newBlock); + } + + ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock) + throws IOException { + dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock, newBlock, nodes, storageIDs); return newBlock; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 7b7db7532936f..a1777962756a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator; +import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -37,18 +38,64 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; -/**************************************************************************** - * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}. - * There are two kinds of StripedDataStreamer, leading streamer and ordinary - * stream. Leading streamer requests a block group from NameNode, unwraps - * it to located blocks and transfers each located block to its corresponding - * ordinary streamer via a blocking queue. - * - ****************************************************************************/ +/** + * This class extends {@link DataStreamer} to support writing striped blocks + * to datanodes. + * A {@link DFSStripedOutputStream} has multiple {@link StripedDataStreamer}s. + * Whenever the streamers need to talk the namenode, only the fastest streamer + * sends an rpc call to the namenode and then populates the result for the + * other streamers. + */ public class StripedDataStreamer extends DataStreamer { + /** + * This class is designed for multiple threads to share a + * {@link MultipleBlockingQueue}. Initially, the queue is empty. The earliest + * thread calling poll populates entries to the queue and the other threads + * will wait for it. Once the entries are populated, all the threads can poll + * their entries. + * + * @param the queue entry type. + */ + static abstract class ConcurrentPoll { + private final MultipleBlockingQueue queue; + + ConcurrentPoll(MultipleBlockingQueue queue) { + this.queue = queue; + } + + T poll(final int i) throws IOException { + for(;;) { + synchronized(queue) { + final T polled = queue.poll(i); + if (polled != null) { // already populated; return polled item. + return polled; + } + if (isReady2Populate()) { + populate(); + return queue.poll(i); + } + } + + // sleep and then retry. + try { + Thread.sleep(100); + } catch(InterruptedException ie) { + throw DFSUtil.toInterruptedIOException( + "Sleep interrupted during poll", ie); + } + } + } + + boolean isReady2Populate() { + return queue.isEmpty(); + } + + abstract void populate() throws IOException; + } + private final Coordinator coordinator; private final int index; - private volatile boolean isFailed; + private volatile boolean failed; StripedDataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, @@ -66,16 +113,12 @@ int getIndex() { return index; } - void setIsFailed(boolean isFailed) { - this.isFailed = isFailed; + void setFailed(boolean failed) { + this.failed = failed; } boolean isFailed() { - return isFailed; - } - - public boolean isLeadingStreamer () { - return index == 0; + return failed; } private boolean isParityStreamer() { @@ -85,81 +128,110 @@ private boolean isParityStreamer() { @Override protected void endBlock() { if (!isParityStreamer()) { - coordinator.putEndBlock(index, block); + coordinator.offerEndBlock(index, block); } super.endBlock(); } @Override - protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) + protected LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes) throws IOException { - if (isLeadingStreamer()) { - if (block != null) { - // set numByte for the previous block group - long bytes = 0; - for (int i = 0; i < NUM_DATA_BLOCKS; i++) { - final ExtendedBlock b = coordinator.getEndBlock(i); - if (b != null) { - StripedBlockUtil.checkBlocks(block, i, b); + final MultipleBlockingQueue followingBlocks + = coordinator.getFollowingBlocks(); + return new ConcurrentPoll(followingBlocks) { + @Override + boolean isReady2Populate() { + return super.isReady2Populate() + && (block == null || coordinator.hasAllEndBlocks()); + } + + @Override + void populate() throws IOException { + getLastException().check(false); + + if (block != null) { + // set numByte for the previous block group + long bytes = 0; + for (int i = 0; i < NUM_DATA_BLOCKS; i++) { + final ExtendedBlock b = coordinator.takeEndBlock(i); + StripedBlockUtil.checkBlocks(index, block, i, b); bytes += b.getNumBytes(); } + block.setNumBytes(bytes); + block.setBlockId(block.getBlockId() - index); } - block.setNumBytes(bytes); - } - putLoactedBlocks(super.locateFollowingBlock(excludedNodes)); - } - - return coordinator.getStripedBlock(index); - } + if (LOG.isDebugEnabled()) { + LOG.debug("locateFollowingBlock: index=" + index + ", block=" + block); + } - void putLoactedBlocks(LocatedBlock lb) throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("Obtained block group " + lb); - } - LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( - (LocatedStripedBlock)lb, - BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); - - // TODO allow write to continue if blocks.length >= NUM_DATA_BLOCKS - assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) : - "Fail to get block group from namenode: blockGroupSize: " + - (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " + - blocks.length; - for (int i = 0; i < blocks.length; i++) { - coordinator.putStripedBlock(i, blocks[i]); - } + final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock( + excludedNodes); + final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock)lb, + BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + + for (int i = 0; i < blocks.length; i++) { + if (!coordinator.getStripedDataStreamer(i).isFailed()) { + if (blocks[i] == null) { + getLastException().set( + new IOException("Failed to get following block, i=" + i)); + } else { + followingBlocks.offer(i, blocks[i]); + } + } + } + } + }.poll(index); } @Override LocatedBlock updateBlockForPipeline() throws IOException { - if (isLeadingStreamer()) { - final LocatedBlock updated = super.updateBlockForPipeline(); - final ExtendedBlock block = updated.getBlock(); - for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { - final LocatedBlock lb = new LocatedBlock(block, null, null, null, - -1, updated.isCorrupt(), null); - lb.setBlockToken(updated.getBlockToken()); - coordinator.putStripedBlock(i, lb); + final MultipleBlockingQueue newBlocks + = coordinator.getNewBlocks(); + return new ConcurrentPoll(newBlocks) { + @Override + void populate() throws IOException { + final ExtendedBlock bg = coordinator.getBlockGroup(); + final LocatedBlock updated = callUpdateBlockForPipeline(bg); + final long newGS = updated.getBlock().getGenerationStamp(); + for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { + final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock(); + if (bi != null) { + final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS), + null, null, null, -1, updated.isCorrupt(), null); + lb.setBlockToken(updated.getBlockToken()); + newBlocks.offer(i, lb); + } else { + final LocatedBlock lb = coordinator.getFollowingBlocks().peek(i); + lb.getBlock().setGenerationStamp(newGS); + } + } } - } - return coordinator.getStripedBlock(index); + }.poll(index); } @Override - ExtendedBlock updatePipeline(long newGS) throws IOException { - if (isLeadingStreamer()) { - final ExtendedBlock newBlock = super.updatePipeline(newGS); - for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { - coordinator.putUpdateBlock(i, new ExtendedBlock(newBlock)); + ExtendedBlock updatePipeline(final long newGS) throws IOException { + final MultipleBlockingQueue updateBlocks + = coordinator.getUpdateBlocks(); + return new ConcurrentPoll(updateBlocks) { + @Override + void populate() throws IOException { + final ExtendedBlock bg = coordinator.getBlockGroup(); + final ExtendedBlock newBG = newBlock(bg, newGS); + final ExtendedBlock updated = callUpdatePipeline(bg, newBG); + for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { + final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock(); + updateBlocks.offer(i, newBlock(bi, updated.getGenerationStamp())); + } } - } - return coordinator.getUpdateBlock(index); + }.poll(index); } @Override public String toString() { - return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0) + return "#" + index + ": failed? " + Boolean.toString(failed).charAt(0) + ", " + super.toString(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java index 34ec06d999894..9aef436a806be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java @@ -103,8 +103,6 @@ public class DfsClientConf { private final int hedgedReadThreadpoolSize; private final int stripedReadThreadpoolSize; - private final int stripedWriteMaxSecondsGetStripedBlock; - private final int stripedWriteMaxSecondsGetEndedBlock; public DfsClientConf(Configuration conf) { @@ -228,13 +226,6 @@ public DfsClientConf(Configuration conf) { Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY + " must be greater than 0."); - - stripedWriteMaxSecondsGetStripedBlock = conf.getInt( - HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_STRIPED_BLOCK_KEY, - HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_STRIPED_BLOCK_DEFAULT); - stripedWriteMaxSecondsGetEndedBlock = conf.getInt( - HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_ENDED_BLOCK_KEY, - HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_ENDED_BLOCK_DEFAULT); } private DataChecksum.Type getChecksumType(Configuration conf) { @@ -518,20 +509,6 @@ public int getStripedReadThreadpoolSize() { return stripedReadThreadpoolSize; } - /** - * @return stripedWriteMaxSecondsGetStripedBlock - */ - public int getStripedWriteMaxSecondsGetStripedBlock() { - return stripedWriteMaxSecondsGetStripedBlock; - } - - /** - * @return stripedWriteMaxSecondsGetEndedBlock - */ - public int getStripedWriteMaxSecondsGetEndedBlock() { - return stripedWriteMaxSecondsGetEndedBlock; - } - /** * @return the shortCircuitConf */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index a29e8e35185ec..579434b67069a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -950,22 +950,22 @@ public String toString() { /** * Check if the information such as IDs and generation stamps in block-i - * match block-0. + * match block-j, where block-i and block-j are in the same group. */ - public static void checkBlocks(ExtendedBlock block0, int i, - ExtendedBlock blocki) throws IOException { + public static void checkBlocks(int j, ExtendedBlock blockj, + int i, ExtendedBlock blocki) throws IOException { - if (!blocki.getBlockPoolId().equals(block0.getBlockPoolId())) { - throw new IOException("Block pool IDs mismatched: block0=" - + block0 + ", block" + i + "=" + blocki); + if (!blocki.getBlockPoolId().equals(blockj.getBlockPoolId())) { + throw new IOException("Block pool IDs mismatched: block" + j + "=" + + blockj + ", block" + i + "=" + blocki); } - if (blocki.getBlockId() - i != block0.getBlockId()) { - throw new IOException("Block IDs mismatched: block0=" - + block0 + ", block" + i + "=" + blocki); + if (blocki.getBlockId() - i != blockj.getBlockId() - j) { + throw new IOException("Block IDs mismatched: block" + j + "=" + + blockj + ", block" + i + "=" + blocki); } - if (blocki.getGenerationStamp() != block0.getGenerationStamp()) { - throw new IOException("Generation stamps mismatched: block0=" - + block0 + ", block" + i + "=" + blocki); + if (blocki.getGenerationStamp() != blockj.getGenerationStamp()) { + throw new IOException("Generation stamps mismatched: block" + j + "=" + + blockj + ", block" + i + "=" + blocki); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index d2e045827188c..8944cde02de79 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -92,6 +92,13 @@ private static byte getByte(long pos) { return (byte)pos; } + @Test(timeout=120000) + public void testDatanodeFailure0() { + final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); + final int dn = 0; + runTest("file" + dn, length, dn); + } + @Test(timeout=120000) public void testDatanodeFailure1() { final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); From 448cb7df676d3c0f5fdc52fbbe736f3b54e519a3 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 19 Jun 2015 11:53:05 -0700 Subject: [PATCH 163/212] HDFS-8543. Erasure Coding: processOverReplicatedBlock() handles striped block. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockManager.java | 151 ++++++++++++++---- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 15 +- .../hdfs/server/balancer/TestBalancer.java | 1 + .../hadoop/hdfs/server/mover/TestMover.java | 1 + .../TestAddOverReplicatedStripedBlocks.java | 116 ++++++++++++++ 6 files changed, 254 insertions(+), 33 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index a710c2efa81f0..a12f361bceeaf 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -311,3 +311,6 @@ HDFS-8254. Avoid assigning a leading streamer in StripedDataStreamer to tolerate datanode failure. (Tsz Wo Nicholas Sze via jing9) + + HDFS-8543. Erasure Coding: processOverReplicatedBlock() handles striped block. + (Walter Su via jing9) 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 48a1b35d45d6e..2533ca54e17cc 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 @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; @@ -3085,10 +3086,30 @@ private void processOverReplicatedBlock(final BlockInfo block, } } } - chooseExcessReplicates(nonExcess, block, replication, - addedNode, delNodeHint, placementPolicies.getPolicy(false)); + chooseExcessReplicates(nonExcess, block, replication, addedNode, + delNodeHint); } + private void chooseExcessReplicates( + final Collection nonExcess, + BlockInfo storedBlock, short replication, + DatanodeDescriptor addedNode, + DatanodeDescriptor delNodeHint) { + assert namesystem.hasWriteLock(); + // first form a rack to datanodes map and + BlockCollection bc = getBlockCollection(storedBlock); + final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( + bc.getStoragePolicyID()); + final List excessTypes = storagePolicy.chooseExcess( + replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); + if (!storedBlock.isStriped()) { + chooseExcessReplicasContiguous(bc, nonExcess, storedBlock, + replication, addedNode, delNodeHint, excessTypes); + } else { + chooseExcessReplicasStriped(bc, nonExcess, storedBlock, delNodeHint, + excessTypes); + } + } /** * We want "replication" replicates for the block, but we now have too many. @@ -3104,20 +3125,13 @@ private void processOverReplicatedBlock(final BlockInfo block, * If no such a node is available, * then pick a node with least free space */ - private void chooseExcessReplicates(final Collection nonExcess, - BlockInfo storedBlock, short replication, - DatanodeDescriptor addedNode, - DatanodeDescriptor delNodeHint, - BlockPlacementPolicy replicator) { - assert namesystem.hasWriteLock(); - // first form a rack to datanodes map and - BlockCollection bc = getBlockCollection(storedBlock); - final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( - bc.getStoragePolicyID()); - final List excessTypes = storagePolicy.chooseExcess( - replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); - - + private void chooseExcessReplicasContiguous(BlockCollection bc, + final Collection nonExcess, + BlockInfo storedBlock, short replication, + DatanodeDescriptor addedNode, + DatanodeDescriptor delNodeHint, + List excessTypes) { + BlockPlacementPolicy replicator = placementPolicies.getPolicy(false); final Map> rackMap = new HashMap<>(); final List moreThanOne = new ArrayList<>(); final List exactlyOne = new ArrayList<>(); @@ -3145,28 +3159,101 @@ private void chooseExcessReplicates(final Collection nonExc moreThanOne, exactlyOne, excessTypes); } firstOne = false; - // adjust rackmap, moreThanOne, and exactlyOne replicator.adjustSetsWithChosenReplica(rackMap, moreThanOne, exactlyOne, cur); - nonExcess.remove(cur); - addToExcessReplicate(cur.getDatanodeDescriptor(), storedBlock); + processChosenExcessReplica(nonExcess, cur, storedBlock); + } + } - // - // The 'excessblocks' tracks blocks until we get confirmation - // that the datanode has deleted them; the only way we remove them - // is when we get a "removeBlock" message. - // - // The 'invalidate' list is used to inform the datanode the block - // should be deleted. Items are removed from the invalidate list - // upon giving instructions to the datanodes. - // - final Block blockToInvalidate = getBlockToInvalidate(storedBlock, cur); - addToInvalidates(blockToInvalidate, cur.getDatanodeDescriptor()); - blockLog.info("BLOCK* chooseExcessReplicates: " - +"({}, {}) is added to invalidated blocks set", cur, storedBlock); + /** + * We want block group has every internal block, but we have redundant + * internal blocks (which have the same index). + * In this method, we delete the redundant internal blocks until only one + * left for each index. + * + * The block placement policy will make sure that the left internal blocks are + * spread across racks and also try hard to pick one with least free space. + */ + private void chooseExcessReplicasStriped(BlockCollection bc, + final Collection nonExcess, + BlockInfo storedBlock, + DatanodeDescriptor delNodeHint, + List excessTypes) { + assert storedBlock instanceof BlockInfoStriped; + BlockInfoStriped sblk = (BlockInfoStriped) storedBlock; + short groupSize = sblk.getTotalBlockNum(); + if (nonExcess.size() <= groupSize) { + return; + } + BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true); + List empty = new ArrayList<>(0); + + // find all duplicated indices + BitSet found = new BitSet(groupSize); //indices found + BitSet duplicated = new BitSet(groupSize); //indices found more than once + HashMap storage2index = new HashMap<>(); + for (DatanodeStorageInfo storage : nonExcess) { + int index = sblk.getStorageBlockIndex(storage); + assert index >= 0; + if (found.get(index)) { + duplicated.set(index); + } + found.set(index); + storage2index.put(storage, index); } + + // use delHint only if delHint is duplicated + final DatanodeStorageInfo delStorageHint = + DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, delNodeHint); + if (delStorageHint != null) { + Integer index = storage2index.get(delStorageHint); + if (index != null && duplicated.get(index)) { + processChosenExcessReplica(nonExcess, delStorageHint, storedBlock); + } + } + + // for each duplicated index, delete some replicas until only one left + for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0; + targetIndex = duplicated.nextSetBit(targetIndex + 1)) { + List candidates = new ArrayList<>(); + for (DatanodeStorageInfo storage : nonExcess) { + int index = storage2index.get(storage); + if (index == targetIndex) { + candidates.add(storage); + } + } + Block internalBlock = new Block(storedBlock); + internalBlock.setBlockId(storedBlock.getBlockId() + targetIndex); + while (candidates.size() > 1) { + DatanodeStorageInfo target = placementPolicy.chooseReplicaToDelete(bc, + internalBlock, (short)1, candidates, empty, excessTypes); + processChosenExcessReplica(nonExcess, target, storedBlock); + candidates.remove(target); + } + duplicated.clear(targetIndex); + } + } + + private void processChosenExcessReplica( + final Collection nonExcess, + final DatanodeStorageInfo chosen, BlockInfo storedBlock) { + nonExcess.remove(chosen); + addToExcessReplicate(chosen.getDatanodeDescriptor(), storedBlock); + // + // The 'excessblocks' tracks blocks until we get confirmation + // that the datanode has deleted them; the only way we remove them + // is when we get a "removeBlock" message. + // + // The 'invalidate' list is used to inform the datanode the block + // should be deleted. Items are removed from the invalidate list + // upon giving instructions to the datanodes. + // + final Block blockToInvalidate = getBlockToInvalidate(storedBlock, chosen); + addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor()); + blockLog.info("BLOCK* chooseExcessReplicates: " + +"({}, {}) is added to invalidated blocks set", chosen, storedBlock); } /** Check if we can use delHint */ 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 b9ded80ac9204..7c9eabf0d36b0 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 @@ -110,6 +110,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; 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; @@ -1968,17 +1969,29 @@ public static ExtendedBlock flushInternal(DFSStripedOutputStream out) } /** - * Verify that blocks in striped block group are on different nodes. + * Verify that blocks in striped block group are on different nodes, and every + * internal blocks exists. */ public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, int groupSize) { for (LocatedBlock lb : lbs.getLocatedBlocks()) { + assert lb instanceof LocatedStripedBlock; HashSet locs = new HashSet<>(); for (DatanodeInfo datanodeInfo : lb.getLocations()) { locs.add(datanodeInfo); } assertEquals(groupSize, lb.getLocations().length); assertEquals(groupSize, locs.size()); + + // verify that every internal blocks exists + int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices(); + assertEquals(groupSize, blockIndices.length); + HashSet found = new HashSet<>(); + for (int index : blockIndices) { + assert index >=0; + found.add(index); + } + assertEquals(groupSize, found.size()); } } } 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 f6475cd82b04d..759eb45771c3f 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 @@ -143,6 +143,7 @@ static void initConfWithRamDisk(Configuration conf, static void initConfWithStripe(Configuration conf) { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); SimulatedFSDataset.setFactory(conf); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index 74f09fdabd1bc..29e8d24383e26 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -379,6 +379,7 @@ static void initConfWithStripe(Configuration conf) { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); Dispatcher.setBlockMoveWaitTime(3000L); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java new file mode 100644 index 0000000000000..eaf3435817b7b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestAddOverReplicatedStripedBlocks { + + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private final Path dirPath = new Path("/striped"); + private Path filePath = new Path(dirPath, "file"); + private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; + private final short GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; + private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int NUM_STRIPE_PER_BLOCK = 1; + private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; + private final int numDNs = GROUP_SIZE + 3; + + @Before + public void setup() throws IOException { + Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + SimulatedFSDataset.setFactory(conf); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + fs.mkdirs(dirPath); + fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testProcessOverReplicatedStripedBlock() throws Exception { + // create a file which has exact one block group to the first GROUP_SIZE DNs + long fileLen = DATA_BLK_NUM * BLOCK_SIZE; + DFSTestUtil.createStripedFile(cluster, filePath, null, 1, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + long gs = bg.getBlock().getGenerationStamp(); + String bpid = bg.getBlock().getBlockPoolId(); + long groupId = bg.getBlock().getBlockId(); + Block blk = new Block(groupId, BLOCK_SIZE, gs); + for (int i = 0; i < GROUP_SIZE; i++) { + blk.setBlockId(groupId + i); + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + } + cluster.triggerBlockReports(); + + // let a internal block be over replicated with 2 redundant blocks. + blk.setBlockId(groupId + 2); + cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid); + cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid); + // let a internal block be over replicated with 1 redundant block. + blk.setBlockId(groupId + 6); + cluster.injectBlocks(numDNs - 1, Arrays.asList(blk), bpid); + + // update blocksMap + cluster.triggerBlockReports(); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // verify that all internal blocks exists + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); + } +} From 8c423a8163bf7e27422e101f3d4768c3ca1a6b88 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 19 Jun 2015 14:07:38 -0700 Subject: [PATCH 164/212] HDFS-8602. Erasure Coding: Client can't read(decode) the EC files which have corrupt blocks. Contributed by Jing Zhao and Kai Sasaki. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 15 +- .../hdfs/TestReadStripedFileWithDecoding.java | 128 ++++++++++++++++-- 3 files changed, 129 insertions(+), 17 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index a12f361bceeaf..2c91dad62dcc8 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -314,3 +314,6 @@ HDFS-8543. Erasure Coding: processOverReplicatedBlock() handles striped block. (Walter Su via jing9) + + HDFS-8602. Erasure Coding: Client can't read(decode) the EC files which have + corrupt blocks. (jing9 and Kai Sasaki) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index a7339b7d16930..878e5e184dcf2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -45,7 +45,6 @@ import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.net.NetUtils; import org.apache.htrace.Span; @@ -340,7 +339,7 @@ private void readOneStripe( private Callable readCell(final BlockReader reader, final DatanodeInfo datanode, final long currentReaderOffset, final long targetReaderOffset, final ByteBufferStrategy strategy, - final int targetLength, + final int targetLength, final ExtendedBlock currentBlock, final Map> corruptedBlockMap) { return new Callable() { @Override @@ -359,7 +358,8 @@ public Void call() throws Exception { } int result = 0; while (result < targetLength) { - int ret = readToBuffer(reader, datanode, strategy, corruptedBlockMap); + int ret = readToBuffer(reader, datanode, strategy, currentBlock, + corruptedBlockMap); if (ret < 0) { throw new IOException("Unexpected EOS from the reader"); } @@ -373,21 +373,22 @@ public Void call() throws Exception { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy readerStrategy, + ExtendedBlock currentBlock, Map> corruptedBlockMap) throws IOException { try { return readerStrategy.doRead(blockReader, 0, 0); } catch (ChecksumException ce) { DFSClient.LOG.warn("Found Checksum error for " - + getCurrentBlock() + " from " + currentNode + + currentBlock + " from " + currentNode + " at " + ce.getPos()); // we want to remember which block replicas we have tried - addIntoCorruptedBlockMap(getCurrentBlock(), currentNode, + addIntoCorruptedBlockMap(currentBlock, currentNode, corruptedBlockMap); throw ce; } catch (IOException e) { DFSClient.LOG.warn("Exception while reading from " - + getCurrentBlock() + " of " + src + " from " + + currentBlock + " of " + src + " from " + currentNode, e); throw e; } @@ -768,7 +769,7 @@ void readChunk(final CompletionService service, Callable readCallable = readCell(blockReaders[chunkIndex], currentNodes[chunkIndex], blockReaderOffsets[chunkIndex], alignedStripe.getOffsetInBlock(), strategy, - chunk.byteBuffer.remaining(), corruptedBlockMap); + chunk.byteBuffer.remaining(), block.getBlock(), corruptedBlockMap); Future request = readingService.submit(readCallable); futures.put(request, chunkIndex); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 0201d071ab275..3125e2e8aa883 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -17,35 +17,37 @@ */ package org.apache.hadoop.hdfs; -import org.apache.hadoop.conf.Configuration; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.*; public class TestReadStripedFileWithDecoding { + static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class); private MiniDFSCluster cluster; - private FileSystem fs; + private DistributedFileSystem fs; @Before public void setup() throws IOException { - Configuration conf = new HdfsConfiguration(); - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()) + .numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null, cellSize); fs = cluster.getFileSystem(); @@ -73,6 +75,112 @@ public void testReadWithDNFailure3() throws IOException { testReadWithDNFailure("/foo", cellSize * dataBlocks, 0); } + /** + * Delete a data block before reading. Verify the decoding works correctly. + */ + @Test + public void testReadCorruptedData() throws IOException { + // create file + final Path file = new Path("/partially_deleted"); + final int length = cellSize * dataBlocks * 2; + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, file, bytes); + + // corrupt the first data block + // find the corresponding data node + int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); + Assert.assertNotEquals(-1, dnIndex); + // find the target block + LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() + .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, + cellSize, dataBlocks, parityBlocks); + // find the target block file + File storageDir = cluster.getInstanceStorageDir(dnIndex, 0); + File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock()); + Assert.assertTrue("Block file does not exist", blkFile.exists()); + // delete the block file + LOG.info("Deliberately removing file " + blkFile.getName()); + Assert.assertTrue("Cannot remove file", blkFile.delete()); + verifyRead(file, length, bytes); + } + + /** + * Corrupt the content of the data block before reading. + */ + @Test + public void testReadCorruptedData2() throws IOException { + // create file + final Path file = new Path("/partially_corrupted"); + final int length = cellSize * dataBlocks * 2; + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, file, bytes); + + // corrupt the first data block + // find the first data node + int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); + Assert.assertNotEquals(-1, dnIndex); + // find the first data block + LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() + .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, + cellSize, dataBlocks, parityBlocks); + // find the first block file + File storageDir = cluster.getInstanceStorageDir(dnIndex, 0); + File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock()); + Assert.assertTrue("Block file does not exist", blkFile.exists()); + // corrupt the block file + LOG.info("Deliberately corrupting file " + blkFile.getName()); + try (FileOutputStream out = new FileOutputStream(blkFile)) { + out.write("corruption".getBytes()); + } + + verifyRead(file, length, bytes); + } + + private int findFirstDataNode(Path file, long length) throws IOException { + BlockLocation[] locs = fs.getFileBlockLocations(file, 0, length); + String name = (locs[0].getNames())[0]; + int dnIndex = 0; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + return dnIndex; + } + dnIndex++; + } + return -1; + } + + private void verifyRead(Path file, int length, byte[] expected) + throws IOException { + // pread + try (FSDataInputStream fsdis = fs.open(file)) { + byte[] buf = new byte[length]; + int readLen = fsdis.read(0, buf, 0, buf.length); + Assert.assertEquals("The fileSize of file should be the same to write size", + length, readLen); + Assert.assertArrayEquals(expected, buf); + } + + // stateful read + ByteBuffer result = ByteBuffer.allocate(length); + ByteBuffer buf = ByteBuffer.allocate(1024); + int readLen = 0; + int ret; + try (FSDataInputStream in = fs.open(file)) { + while ((ret = in.read(buf)) >= 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } + Assert.assertEquals("The length of file should be the same to write size", + length, readLen); + Assert.assertArrayEquals(expected, result.array()); + } + private void testReadWithDNFailure(String file, int fileSize, int startOffsetInFile) throws IOException { final int failedDNIdx = 2; From d920cdab91781e547932234d52807ba5a59eb120 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 23 Jun 2015 22:29:28 -0700 Subject: [PATCH 165/212] HDFS-8567. Erasure Coding: SafeMode handles file smaller than a full stripe. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../blockmanagement/BlockInfoStriped.java | 18 +++ .../server/blockmanagement/BlockManager.java | 30 ++-- .../hdfs/server/namenode/FSNamesystem.java | 2 +- .../org/apache/hadoop/hdfs/TestSafeMode.java | 11 -- .../hdfs/TestSafeModeWithStripedFile.java | 150 ++++++++++++++++++ 6 files changed, 181 insertions(+), 33 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2c91dad62dcc8..7f624252ec40c 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -317,3 +317,6 @@ HDFS-8602. Erasure Coding: Client can't read(decode) the EC files which have corrupt blocks. (jing9 and Kai Sasaki) + + HDFS-8567. Erasure Coding: SafeMode handles file smaller than a full stripe. + (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 4cede91281990..07e29f8a7132a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -73,6 +73,24 @@ public short getParityBlockNum() { return (short) this.schema.getNumParityUnits(); } + /** + * If the block is committed/completed and its length is less than a full + * stripe, it returns the the number of actual data blocks. + * Otherwise it returns the number of data units specified by schema. + */ + public short getRealDataBlockNum() { + if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) { + return (short) Math.min(getDataBlockNum(), + (getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1); + } else { + return getDataBlockNum(); + } + } + + public short getRealTotalBlockNum() { + return (short) (getRealDataBlockNum() + getParityBlockNum()); + } + public ECSchema getSchema() { return schema; } 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 2533ca54e17cc..d57e6f1c810f1 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 @@ -592,13 +592,16 @@ public int getMaxReplicationStreams() { } public int getDefaultStorageNum(BlockInfo block) { - return block.isStriped() ? - ((BlockInfoStriped) block).getTotalBlockNum() : defaultReplication; + if (block.isStriped()) { + return ((BlockInfoStriped) block).getRealTotalBlockNum(); + } else { + return defaultReplication; + } } public short getMinStorageNum(BlockInfo block) { if (block.isStriped()) { - return getStripedDataBlockNum(block); + return ((BlockInfoStriped) block).getRealDataBlockNum(); } else { return minReplication; } @@ -707,7 +710,7 @@ private BlockInfo completeBlock(final BlockCollection bc, // OP_CLOSE edit on the standby). namesystem.adjustSafeModeBlockTotals(0, 1); final int minStorage = curBlock.isStriped() ? - ((BlockInfoStriped) curBlock).getDataBlockNum() : minReplication; + ((BlockInfoStriped) curBlock).getRealDataBlockNum() : minReplication; namesystem.incrementSafeBlockCount( Math.min(numNodes, minStorage), curBlock); @@ -3854,27 +3857,12 @@ boolean isNeededReplication(BlockInfo storedBlock, int expected, int current) { public short getExpectedReplicaNum(BlockCollection bc, BlockInfo block) { if (block.isStriped()) { - return (short) (getStripedDataBlockNum(block) + - ((BlockInfoStriped) block).getParityBlockNum()); + return ((BlockInfoStriped) block).getRealTotalBlockNum(); } else { return bc.getPreferredBlockReplication(); } } - - short getStripedDataBlockNum(BlockInfo block) { - assert block.isStriped(); - final BlockInfoStriped sblock = (BlockInfoStriped) block; - short dataBlockNum = sblock.getDataBlockNum(); - if (sblock.isComplete() || - sblock.getBlockUCState() == BlockUCState.COMMITTED) { - // if the sblock is committed/completed and its length is less than a - // full stripe, the minimum storage number needs to be adjusted - dataBlockNum = (short) Math.min(dataBlockNum, - (sblock.getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1); - } - return dataBlockNum; - } - + public long getMissingBlocksCount() { // not locking return this.neededReplications.getCorruptBlockSize(); 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 1dd0a432d6aaa..e5483b62494b2 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 @@ -4878,7 +4878,7 @@ private synchronized void setBlockTotal(int total) { private synchronized void incrementSafeBlockCount(short storageNum, BlockInfo storedBlock) { final int safe = storedBlock.isStriped() ? - ((BlockInfoStriped) storedBlock).getDataBlockNum() : safeReplication; + ((BlockInfoStriped) storedBlock).getRealDataBlockNum() : safeReplication; if (storageNum == safe) { this.blockSafe++; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java index a43e37173f566..1ea89515de73e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java @@ -553,17 +553,6 @@ public void testSafeModeWhenZeroBlockLocations() throws IOException { } } - //TODO : test should be added to check safeMode with stripedBloks after stripedBlock related functions have been added in class MiniDFSCluster - @Test - public void testSafeModeWithCorruptSripedBlock() throws IOException { - try { - - } finally { - if(fs != null) fs.close(); - if(cluster!= null) cluster.shutdown(); - } - } - void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOException { FileStatus stat = fs.getFileStatus(fileName); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java new file mode 100644 index 0000000000000..6f0bc71ad5baa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java @@ -0,0 +1,150 @@ +/** + * 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 com.google.common.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestSafeModeWithStripedFile { + + static final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; + static final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; + static final int numDNs = DATA_BLK_NUM + PARITY_BLK_NUM; + static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + static final int blockSize = cellSize * 2; + + static MiniDFSCluster cluster; + static Configuration conf; + + @Before + public void setup() throws IOException { + conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().createErasureCodingZone("/", + null, cellSize); + cluster.waitActive(); + + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testStripedFile0() throws IOException { + doTest(cellSize, 1); + } + + @Test + public void testStripedFile1() throws IOException { + doTest(cellSize * 5, 5); + } + + /** + * This util writes a small block group whose size is given by caller. + * Then write another 2 full stripe blocks. + * Then shutdown all DNs and start again one by one. and verify the safemode + * status accordingly. + * + * @param smallSize file size of the small block group + * @param minStorages minimum replicas needed by the block so it can be safe + */ + private void doTest(int smallSize, int minStorages) throws IOException { + FileSystem fs = cluster.getFileSystem(); + // add 1 block + byte[] data = StripedFileTestUtil.generateBytes(smallSize); + Path smallFilePath = new Path("/testStripedFile_" + smallSize); + DFSTestUtil.writeFile(fs, smallFilePath, data); + + // If we only have 1 block, NN won't enter safemode in the first place + // because the threshold is 0 blocks. + // So we need to add another 2 blocks. + int bigSize = blockSize * DATA_BLK_NUM * 2; + Path bigFilePath = new Path("/testStripedFile_" + bigSize); + data = StripedFileTestUtil.generateBytes(bigSize); + DFSTestUtil.writeFile(fs, bigFilePath, data); + // now we have 3 blocks. NN needs 2 blocks to reach the threshold 0.9 of + // total blocks 3. + + // stopping all DNs + List dnprops = Lists.newArrayList(); + LocatedBlocks lbs = cluster.getNameNodeRpc() + .getBlockLocations(smallFilePath.toString(), 0, smallSize); + DatanodeInfo[] locations = lbs.get(0).getLocations(); + for (DatanodeInfo loc : locations) { + // keep the DNs that have smallFile in the head of dnprops + dnprops.add(cluster.stopDataNode(loc.getName())); + } + for (int i = 0; i < numDNs - locations.length; i++) { + dnprops.add(cluster.stopDataNode(0)); + } + + cluster.restartNameNode(0); + NameNode nn = cluster.getNameNode(); + assertTrue(cluster.getNameNode().isInSafeMode()); + assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn)); + + // the block of smallFile doesn't reach minStorages, + // so the safe blocks count doesn't increment. + for (int i = 0; i < minStorages - 1; i++) { + cluster.restartDataNode(dnprops.remove(0)); + cluster.triggerBlockReports(); + assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn)); + } + + // the block of smallFile reaches minStorages, + // so the safe blocks count increment. + cluster.restartDataNode(dnprops.remove(0)); + cluster.triggerBlockReports(); + assertEquals(1, NameNodeAdapter.getSafeModeSafeBlocks(nn)); + + // the 2 blocks of bigFile need DATA_BLK_NUM storages to be safe + for (int i = minStorages; i < DATA_BLK_NUM - 1; i++) { + cluster.restartDataNode(dnprops.remove(0)); + cluster.triggerBlockReports(); + assertTrue(nn.isInSafeMode()); + } + + cluster.restartDataNode(dnprops.remove(0)); + cluster.triggerBlockReports(); + assertFalse(nn.isInSafeMode()); + } + +} From 093907d728f417d9ed06d16d1a7a4afa5f547f03 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 24 Jun 2015 23:39:05 +0800 Subject: [PATCH 166/212] HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers multiple times. Contributed by Kai Sasaki --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hadoop/hdfs/DFSStripedOutputStream.java | 27 ++++++++++--------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 7f624252ec40c..f88042550a27f 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -320,3 +320,6 @@ HDFS-8567. Erasure Coding: SafeMode handles file smaller than a full stripe. (Walter Su via jing9) + + HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers + multiple times. (Kai Sasaki via szetszwo) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 1068b3752ca25..cd7f6d411020c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -494,21 +494,24 @@ boolean isClosed() { @Override protected void closeThreads(boolean force) throws IOException { final MultipleIOException.Builder b = new MultipleIOException.Builder(); - for (StripedDataStreamer streamer : streamers) { - try { - streamer.close(force); - streamer.join(); - streamer.closeSocket(); - } catch(Exception e) { + try { + for (StripedDataStreamer streamer : streamers) { try { - handleStreamerFailure("force=" + force, e); - } catch(IOException ioe) { - b.add(ioe); + streamer.close(force); + streamer.join(); + streamer.closeSocket(); + } catch (Exception e) { + try { + handleStreamerFailure("force=" + force, e); + } catch (IOException ioe) { + b.add(ioe); + } + } finally { + streamer.setSocketToNull(); } - } finally { - streamer.setSocketToNull(); - setClosed(); } + } finally { + setClosed(); } final IOException ioe = b.build(); if (ioe != null) { From 0b7af27b9a369d6abdb1fb6c216f50692267f3f4 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Tue, 30 Jun 2015 11:02:13 +0530 Subject: [PATCH 167/212] HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in double Audit log entries (Contributed by Vinayakumar B) --- .../hadoop/hdfs/protocol/LocatedBlocks.java | 26 ++++++++++++++++-- .../hadoop/hdfs/web/JsonUtilClient.java | 2 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../org/apache/hadoop/hdfs/DFSClient.java | 12 +++++---- .../apache/hadoop/hdfs/DFSInputStream.java | 27 ++++++++++++------- .../hadoop/hdfs/DFSStripedInputStream.java | 8 +++--- .../hadoop/hdfs/protocolPB/PBHelper.java | 11 ++++++-- .../server/blockmanagement/BlockManager.java | 14 ++++++---- .../namenode/FSDirStatAndListingOp.java | 6 ++--- .../hdfs/server/namenode/FSNamesystem.java | 4 ++- .../hadoop-hdfs/src/main/proto/hdfs.proto | 4 +++ .../hadoop/hdfs/TestDFSClientRetries.java | 4 +-- .../hdfs/TestDFSStripedInputStream.java | 8 +++--- .../org/apache/hadoop/hdfs/TestDFSUtil.java | 2 +- 14 files changed, 93 insertions(+), 38 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java index e48969776a6ca..735e7b2ec28e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileEncryptionInfo; +import org.apache.hadoop.io.erasurecode.ECSchema; /** * Collection of blocks with their locations and the file length. @@ -37,6 +38,8 @@ public class LocatedBlocks { private final LocatedBlock lastLocatedBlock; private final boolean isLastBlockComplete; private final FileEncryptionInfo fileEncryptionInfo; + private final ECSchema ecSchema; + private final int stripeCellSize; public LocatedBlocks() { fileLength = 0; @@ -45,17 +48,22 @@ public LocatedBlocks() { lastLocatedBlock = null; isLastBlockComplete = false; fileEncryptionInfo = null; + ecSchema = null; + stripeCellSize = 0; } public LocatedBlocks(long flength, boolean isUnderConstuction, - List blks, LocatedBlock lastBlock, - boolean isLastBlockCompleted, FileEncryptionInfo feInfo) { + List blks, LocatedBlock lastBlock, + boolean isLastBlockCompleted, FileEncryptionInfo feInfo, + ECSchema ecSchema, int stripeCellSize) { fileLength = flength; blocks = blks; underConstruction = isUnderConstuction; this.lastLocatedBlock = lastBlock; this.isLastBlockComplete = isLastBlockCompleted; this.fileEncryptionInfo = feInfo; + this.ecSchema = ecSchema; + this.stripeCellSize = stripeCellSize; } /** @@ -111,6 +119,20 @@ public FileEncryptionInfo getFileEncryptionInfo() { return fileEncryptionInfo; } + /** + * @return The ECSchema for ErasureCoded file, null otherwise. + */ + public ECSchema getECSchema() { + return ecSchema; + } + + /** + * @return Stripe Cell size for ErasureCoded file, 0 otherwise. + */ + public int getStripeCellSize() { + return stripeCellSize; + } + /** * Find block containing specified offset. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java index 5635c1b7d803a..11e221be5d280 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java @@ -479,7 +479,7 @@ static LocatedBlocks toLocatedBlocks( (Map) m.get("lastLocatedBlock")); final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete"); return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks, - lastLocatedBlock, isLastBlockComplete, null); + lastLocatedBlock, isLastBlockComplete, null, null, 0); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index f88042550a27f..7e10753a14d40 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -323,3 +323,6 @@ HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers multiple times. (Kai Sasaki via szetszwo) + + HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in + double Audit log entries (vinayakumarb) \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index a2a622157c859..87b9f906dcacc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1192,15 +1192,17 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) // Get block info from namenode TraceScope scope = getPathTraceScope("newDFSInputStream", src); try { - HdfsFileStatus fileInfo = getFileInfo(src); - if (fileInfo != null) { - ECSchema schema = fileInfo.getECSchema(); + LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0); + if (locatedBlocks != null) { + ECSchema schema = locatedBlocks.getECSchema(); if (schema != null) { return new DFSStripedInputStream(this, src, verifyChecksum, schema, - fileInfo.getStripeCellSize()); + locatedBlocks.getStripeCellSize(), locatedBlocks); } + return new DFSInputStream(this, src, verifyChecksum, locatedBlocks); + } else { + throw new IOException("Cannot open filename " + src); } - return new DFSInputStream(this, src, verifyChecksum); } finally { scope.close(); } 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 1b5705a8b23d7..5b10ffe45ad21 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 @@ -253,24 +253,28 @@ void addToDeadNodes(DatanodeInfo dnInfo) { deadNodes.put(dnInfo, dnInfo); } - DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum - ) throws IOException, UnresolvedLinkException { + DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, + LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException { this.dfsClient = dfsClient; this.verifyChecksum = verifyChecksum; this.src = src; synchronized (infoLock) { this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy(); } - openInfo(); + this.locatedBlocks = locatedBlocks; + openInfo(false); } /** * Grab the open-file info from namenode + * @param refreshLocatedBlocks whether to re-fetch locatedblocks */ - void openInfo() throws IOException, UnresolvedLinkException { + void openInfo(boolean refreshLocatedBlocks) throws IOException, + UnresolvedLinkException { final DfsClientConf conf = dfsClient.getConf(); synchronized(infoLock) { - lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength(); + lastBlockBeingWrittenLength = + fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks); int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength(); while (retriesForLastBlockLength > 0) { // Getting last block length as -1 is a special case. When cluster @@ -282,7 +286,8 @@ void openInfo() throws IOException, UnresolvedLinkException { + "Datanodes might not have reported blocks completely." + " Will retry for " + retriesForLastBlockLength + " times"); waitFor(conf.getRetryIntervalForGetLastBlockLength()); - lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength(); + lastBlockBeingWrittenLength = + fetchLocatedBlocksAndGetLastBlockLength(true); } else { break; } @@ -303,8 +308,12 @@ private void waitFor(int waitTime) throws IOException { } } - private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException { - final LocatedBlocks newInfo = dfsClient.getLocatedBlocks(src, 0); + private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh) + throws IOException { + LocatedBlocks newInfo = locatedBlocks; + if (locatedBlocks == null || refresh) { + newInfo = dfsClient.getLocatedBlocks(src, 0); + } if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("newInfo = " + newInfo); } @@ -1015,7 +1024,7 @@ private DNAddrPair chooseDataNode(LocatedBlock block, } catch (InterruptedException iex) { } deadNodes.clear(); //2nd option is to remove only nodes[blockId] - openInfo(); + openInfo(true); block = refreshLocatedBlock(block); failures++; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 878e5e184dcf2..a71da931dae33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -24,6 +24,7 @@ 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.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; @@ -139,9 +140,10 @@ boolean include(long pos) { private final CompletionService readingService; private ReaderRetryPolicy retry; - DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, - ECSchema schema, int cellSize) throws IOException { - super(dfsClient, src, verifyChecksum); + DFSStripedInputStream(DFSClient dfsClient, String src, + boolean verifyChecksum, ECSchema schema, int cellSize, + LocatedBlocks locatedBlocks) throws IOException { + super(dfsClient, src, verifyChecksum, locatedBlocks); assert schema != null; this.schema = schema; 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 7ee6112506b1d..6baa005c6792f 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 @@ -1338,8 +1338,9 @@ public static LocatedBlocks convert(LocatedBlocksProto lb) { lb.hasLastBlock() ? PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null, lb.getIsLastBlockComplete(), - lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : - null); + lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null, + lb.hasECSchema() ? convertECSchema(lb.getECSchema()) : null, + lb.hasStripeCellSize() ? lb.getStripeCellSize() : 0); } public static LocatedBlocksProto convert(LocatedBlocks lb) { @@ -1355,6 +1356,12 @@ public static LocatedBlocksProto convert(LocatedBlocks lb) { if (lb.getFileEncryptionInfo() != null) { builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo())); } + if (lb.getECSchema() != null) { + builder.setECSchema(convertECSchema(lb.getECSchema())); + } + if (lb.getStripeCellSize() != 0) { + builder.setStripeCellSize(lb.getStripeCellSize()); + } return builder.setFileLength(lb.getFileLength()) .setUnderConstruction(lb.isUnderConstruction()) .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks())) 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 d57e6f1c810f1..dcb0be775d497 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 @@ -938,14 +938,18 @@ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, final long fileSizeExcludeBlocksUnderConstruction, final boolean isFileUnderConstruction, final long offset, final long length, final boolean needBlockToken, - final boolean inSnapshot, FileEncryptionInfo feInfo) + final boolean inSnapshot, FileEncryptionInfo feInfo, + ErasureCodingZone ecZone) throws IOException { assert namesystem.hasReadLock(); + final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; + final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; if (blocks == null) { return null; } else if (blocks.length == 0) { return new LocatedBlocks(0, isFileUnderConstruction, - Collections.emptyList(), null, false, feInfo); + Collections. emptyList(), null, false, feInfo, schema, + cellSize); } else { if (LOG.isDebugEnabled()) { LOG.debug("blocks = " + java.util.Arrays.asList(blocks)); @@ -968,9 +972,9 @@ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, fileSizeExcludeBlocksUnderConstruction, mode); isComplete = true; } - return new LocatedBlocks( - fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction, - locatedblocks, lastlb, isComplete, feInfo); + return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction, + isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo, + schema, cellSize); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index b18c2a37c7681..1e4ee83f8dc38 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -445,6 +445,8 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( final boolean isEncrypted; final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); + final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + fsd.getFSNamesystem(), iip); if (node.isFile()) { final INodeFile fileNode = node.asFile(); size = fileNode.computeFileSize(snapshot); @@ -458,7 +460,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks( fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false, - inSnapshot, feInfo); + inSnapshot, feInfo, ecZone); if (loc == null) { loc = new LocatedBlocks(); } @@ -469,8 +471,6 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( - fsd.getFSNamesystem(), iip); final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; 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 e5483b62494b2..a238eeeefbaad 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 @@ -1873,10 +1873,12 @@ private GetBlockLocationsResult getBlockLocationsInt( final FileEncryptionInfo feInfo = FSDirectory.isReservedRawName(srcArg) ? null : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip); + final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + this, iip); final LocatedBlocks blocks = blockManager.createLocatedBlocks( inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset, - length, needBlockToken, iip.isSnapshot(), feInfo); + length, needBlockToken, iip.isSnapshot(), feInfo, ecZone); // Set caching information for the located blocks. for (LocatedBlock lb : blocks.getLocatedBlocks()) { 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 dd48d7fdb5e86..e1f944fd72567 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -301,6 +301,10 @@ message LocatedBlocksProto { optional LocatedBlockProto lastBlock = 4; required bool isLastBlockComplete = 5; optional FileEncryptionInfoProto fileEncryptionInfo = 6; + + // Optional field for erasure coding + optional ECSchemaProto eCSchema = 7; + optional uint32 stripeCellSize = 8; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java index 4918dee4c41f2..55a335f988f97 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java @@ -343,7 +343,7 @@ public void testFailuresArePerOperation() throws Exception // we're starting a new operation on the user level. doAnswer(new FailNTimesAnswer(preSpyNN, maxBlockAcquires)) .when(spyNN).getBlockLocations(anyString(), anyLong(), anyLong()); - is.openInfo(); + is.openInfo(true); // Seek to beginning forces a reopen of the BlockReader - otherwise it'll // just keep reading on the existing stream and the fact that we've poisoned // the block info won't do anything. @@ -496,7 +496,7 @@ private LocatedBlocks makeBadBlockList(LocatedBlocks goodBlockList) { badBlocks.add(badLocatedBlock); return new LocatedBlocks(goodBlockList.getFileLength(), false, badBlocks, null, true, - null); + null, null, 0); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index b29d58238465b..c520d2c770d80 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -101,7 +101,7 @@ public void testRefreshBlock() throws Exception { LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema, CELLSIZE); + filePath.toString(), false, schema, CELLSIZE, null); List lbList = lbs.getLocatedBlocks(); for (LocatedBlock aLbList : lbList) { @@ -153,7 +153,7 @@ public void testPread() throws Exception { } } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema, CELLSIZE); + filePath.toString(), false, schema, CELLSIZE, null); int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102, CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102, @@ -195,7 +195,7 @@ public void testPreadWithDNFailure() throws Exception { } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, - ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE); + ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE, null); int readSize = BLOCK_GROUP_SIZE; byte[] readBuffer = new byte[readSize]; byte[] expected = new byte[readSize]; @@ -293,7 +293,7 @@ private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), - false, schema, CELLSIZE); + false, schema, CELLSIZE, null); byte[] expected = new byte[fileSize]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index a821c30b2e876..4233a1ce238ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -110,7 +110,7 @@ public void testLocatedBlocks2Locations() { l2.setCorrupt(true); List ls = Arrays.asList(l1, l2); - LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null); + LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null, 0); BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs); From ee01a09500224136464f2c3e0a5d9ba53242d93f Mon Sep 17 00:00:00 2001 From: yliu Date: Mon, 6 Jul 2015 19:14:18 +0800 Subject: [PATCH 168/212] HDFS-8684. Erasure Coding: fix some block number calculation for striped block. (yliu) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 ++++- .../hdfs/server/blockmanagement/BlockManager.java | 11 ++++------- .../apache/hadoop/hdfs/server/namenode/INodeFile.java | 6 +----- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 7e10753a14d40..8f720fc5589ca 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -325,4 +325,7 @@ multiple times. (Kai Sasaki via szetszwo) HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in - double Audit log entries (vinayakumarb) \ No newline at end of file + double Audit log entries (vinayakumarb) + + HDFS-8684. Erasure Coding: fix some block number calculation for striped + block. (yliu) 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 dcb0be775d497..c98bdb04649f7 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 @@ -3109,12 +3109,12 @@ private void chooseExcessReplicates( bc.getStoragePolicyID()); final List excessTypes = storagePolicy.chooseExcess( replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); - if (!storedBlock.isStriped()) { - chooseExcessReplicasContiguous(bc, nonExcess, storedBlock, - replication, addedNode, delNodeHint, excessTypes); - } else { + if (storedBlock.isStriped()) { chooseExcessReplicasStriped(bc, nonExcess, storedBlock, delNodeHint, excessTypes); + } else { + chooseExcessReplicasContiguous(bc, nonExcess, storedBlock, + replication, addedNode, delNodeHint, excessTypes); } } @@ -3191,9 +3191,6 @@ private void chooseExcessReplicasStriped(BlockCollection bc, assert storedBlock instanceof BlockInfoStriped; BlockInfoStriped sblk = (BlockInfoStriped) storedBlock; short groupSize = sblk.getTotalBlockNum(); - if (nonExcess.size() <= groupSize) { - return; - } BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true); List empty = new ArrayList<>(0); 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 46880011b66da..b2b0fea8258a2 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 @@ -809,13 +809,9 @@ public final long computeFileSize(boolean includesLastUcBlock, if (!includesLastUcBlock) { size = 0; } else if (usePreferredBlockSize4LastUcBlock) { - // Striped blocks keeps block group which counts - // (data blocks num + parity blocks num). When you - // count actual used size by BlockInfoStripedUC must - // be multiplied by these blocks number. BlockInfoStripedUnderConstruction blockInfoStripedUC = (BlockInfoStripedUnderConstruction) blockInfos[last]; - size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum(); + size = getPreferredBlockSize() * blockInfoStripedUC.getDataBlockNum(); } } //sum other blocks From 2c494a843699b478039f41336cf47bd4c635eb76 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 6 Jul 2015 16:39:47 -0700 Subject: [PATCH 169/212] HDFS-8461. Erasure coding: fix priority level of UnderReplicatedBlocks for striped block. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../UnderReplicatedBlocks.java | 53 +++++- .../blockmanagement/BlockManagerTestUtil.java | 8 + .../TestUnderReplicatedBlockQueues.java | 62 +++++++ .../namenode/TestRecoverStripedBlocks.java | 151 ++++++++++++------ 5 files changed, 216 insertions(+), 61 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8f720fc5589ca..58b91b60b747d 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -329,3 +329,6 @@ HDFS-8684. Erasure Coding: fix some block number calculation for striped block. (yliu) + + HDFS-8461. Erasure coding: fix priority level of UnderReplicatedBlocks for + striped block. (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java index f9bce268c4ff1..47afb05e9f1ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java @@ -34,7 +34,7 @@ * *

* The policy for choosing which priority to give added blocks - * is implemented in {@link #getPriority(int, int, int)}. + * is implemented in {@link #getPriority(BlockInfo, int, int, int)}. *

*

The queue order is as follows:

*
    @@ -144,14 +144,28 @@ synchronized boolean contains(BlockInfo block) { * @param expectedReplicas expected number of replicas of the block * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1) */ - private int getPriority(int curReplicas, + private int getPriority(BlockInfo block, + int curReplicas, int decommissionedReplicas, int expectedReplicas) { assert curReplicas >= 0 : "Negative replicas!"; if (curReplicas >= expectedReplicas) { // Block has enough copies, but not enough racks return QUEUE_REPLICAS_BADLY_DISTRIBUTED; - } else if (curReplicas == 0) { + } + if (block.isStriped()) { + BlockInfoStriped sblk = (BlockInfoStriped) block; + return getPriorityStriped(curReplicas, decommissionedReplicas, + sblk.getRealDataBlockNum(), sblk.getParityBlockNum()); + } else { + return getPriorityContiguous(curReplicas, decommissionedReplicas, + expectedReplicas); + } + } + + private int getPriorityContiguous(int curReplicas, int decommissionedReplicas, + int expectedReplicas) { + if (curReplicas == 0) { // If there are zero non-decommissioned replicas but there are // some decommissioned replicas, then assign them highest priority if (decommissionedReplicas > 0) { @@ -160,7 +174,7 @@ private int getPriority(int curReplicas, //all we have are corrupt blocks return QUEUE_WITH_CORRUPT_BLOCKS; } else if (curReplicas == 1) { - //only on replica -risk of loss + // only one replica, highest risk of loss // highest priority return QUEUE_HIGHEST_PRIORITY; } else if ((curReplicas * 3) < expectedReplicas) { @@ -173,6 +187,27 @@ private int getPriority(int curReplicas, } } + private int getPriorityStriped(int curReplicas, int decommissionedReplicas, + short dataBlkNum, short parityBlkNum) { + if (curReplicas < dataBlkNum) { + // There are some replicas on decommissioned nodes so it's not corrupted + if (curReplicas + decommissionedReplicas >= dataBlkNum) { + return QUEUE_HIGHEST_PRIORITY; + } + return QUEUE_WITH_CORRUPT_BLOCKS; + } else if (curReplicas == dataBlkNum) { + // highest risk of loss, highest priority + return QUEUE_HIGHEST_PRIORITY; + } else if ((curReplicas - dataBlkNum) * 3 < parityBlkNum + 1) { + // can only afford one replica loss + // this is considered very under-replicated + return QUEUE_VERY_UNDER_REPLICATED; + } else { + // add to the normal queue for under replicated blocks + return QUEUE_UNDER_REPLICATED; + } + } + /** add a block to a under replication queue according to its priority * @param block a under replication block * @param curReplicas current number of replicas of the block @@ -185,7 +220,7 @@ synchronized boolean add(BlockInfo block, int decomissionedReplicas, int expectedReplicas) { assert curReplicas >= 0 : "Negative replicas!"; - int priLevel = getPriority(curReplicas, decomissionedReplicas, + int priLevel = getPriority(block, curReplicas, decomissionedReplicas, expectedReplicas); if(priorityQueues.get(priLevel).add(block)) { if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS && @@ -208,7 +243,7 @@ synchronized boolean remove(BlockInfo block, int oldReplicas, int decommissionedReplicas, int oldExpectedReplicas) { - int priLevel = getPriority(oldReplicas, + int priLevel = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas); boolean removedBlock = remove(block, priLevel); @@ -282,8 +317,10 @@ synchronized void update(BlockInfo block, int curReplicas, int curReplicasDelta, int expectedReplicasDelta) { int oldReplicas = curReplicas-curReplicasDelta; int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta; - int curPri = getPriority(curReplicas, decommissionedReplicas, curExpectedReplicas); - int oldPri = getPriority(oldReplicas, decommissionedReplicas, oldExpectedReplicas); + int curPri = getPriority(block, curReplicas, decommissionedReplicas, + curExpectedReplicas); + int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, + oldExpectedReplicas); if(NameNode.stateChangeLog.isDebugEnabled()) { NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + block + 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 e25ee31643e76..64d80bdbe269c 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 @@ -306,4 +306,12 @@ public static void recheckDecommissionState(DatanodeManager dm) throws ExecutionException, InterruptedException { dm.getDecomManager().runMonitor(); } + + /** + * add block to the replicateBlocks queue of the Datanode + */ + public static void addBlockToBeReplicated(DatanodeDescriptor node, + Block block, DatanodeStorageInfo[] targets) { + node.addBlockToBeReplicated(block, targets); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java index de36e077aea81..0f419ef550fed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java @@ -19,6 +19,9 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -28,10 +31,21 @@ public class TestUnderReplicatedBlockQueues { + private final ECSchema ecSchema = + ErasureCodingSchemaManager.getSystemDefaultSchema(); + private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private BlockInfo genBlockInfo(long id) { return new BlockInfoContiguous(new Block(id), (short) 3); } + private BlockInfo genStripedBlockInfo(long id, long numBytes) { + BlockInfoStriped sblk = new BlockInfoStriped(new Block(id), ecSchema, + CELLSIZE); + sblk.setNumBytes(numBytes); + return sblk; + } + /** * Test that adding blocks with different replication counts puts them * into different queues @@ -85,6 +99,54 @@ public void testBlockPriorities() throws Throwable { assertEquals(2, queues.getCorruptReplOneBlockSize()); } + @Test + public void testStripedBlockPriorities() throws Throwable { + int dataBlkNum = ecSchema.getNumDataUnits(); + int parityBlkNUm = ecSchema.getNumParityUnits(); + doTestStripedBlockPriorities(1, parityBlkNUm); + doTestStripedBlockPriorities(dataBlkNum, parityBlkNUm); + } + + private void doTestStripedBlockPriorities(int dataBlkNum, int parityBlkNum) + throws Throwable { + int groupSize = dataBlkNum + parityBlkNum; + long numBytes = CELLSIZE * dataBlkNum; + UnderReplicatedBlocks queues = new UnderReplicatedBlocks(); + + // add a striped block which been left NUM_DATA_BLOCKS internal blocks + BlockInfo block1 = genStripedBlockInfo(-100, numBytes); + assertAdded(queues, block1, dataBlkNum, 0, groupSize); + assertEquals(1, queues.getUnderReplicatedBlockCount()); + assertEquals(1, queues.size()); + assertInLevel(queues, block1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY); + + // add a striped block which been left NUM_DATA_BLOCKS+1 internal blocks + BlockInfo block2 = genStripedBlockInfo(-200, numBytes); + assertAdded(queues, block2, dataBlkNum + 1, 0, groupSize); + assertEquals(2, queues.getUnderReplicatedBlockCount()); + assertEquals(2, queues.size()); + assertInLevel(queues, block2, + UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED); + + // add a striped block which been left NUM_DATA_BLOCKS+2 internal blocks + BlockInfo block3 = genStripedBlockInfo(-300, numBytes); + assertAdded(queues, block3, dataBlkNum + 2, 0, groupSize); + assertEquals(3, queues.getUnderReplicatedBlockCount()); + assertEquals(3, queues.size()); + assertInLevel(queues, block3, + UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED); + + // add a corrupted block + BlockInfo block_corrupt = genStripedBlockInfo(-400, numBytes); + assertEquals(0, queues.getCorruptBlockSize()); + assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize); + assertEquals(4, queues.size()); + assertEquals(3, queues.getUnderReplicatedBlockCount()); + assertEquals(1, queues.getCorruptBlockSize()); + assertInLevel(queues, block_corrupt, + UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS); + } + private void assertAdded(UnderReplicatedBlocks queues, BlockInfo block, int curReplicas, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index ca4fbbc99cc54..3134373dab4df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -32,29 +32,26 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; -import org.junit.After; -import org.junit.Before; import org.junit.Test; - -import java.io.IOException; import java.util.List; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class TestRecoverStripedBlocks { private final short GROUP_SIZE = - NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS; + NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; private MiniDFSCluster cluster; private final Path dirPath = new Path("/dir"); private Path filePath = new Path(dirPath, "file"); + private int maxReplicationStreams = + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT; - @Before - public void setup() throws IOException { - final Configuration conf = new HdfsConfiguration(); + private void initConf(Configuration conf) { // Large value to make sure the pending replication request can stay in // DatanodeDescriptor.replicateBlocks before test timeout. conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100); @@ -62,63 +59,111 @@ public void setup() throws IOException { // chooseUnderReplicatedBlocks at once. conf.setInt( DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5); + } - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE + 1) - .build(); - cluster.waitActive(); + @Test + public void testMissingStripedBlock() throws Exception { + doTestMissingStripedBlock(1, 0); } - @After - public void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } + @Test + public void testMissingStripedBlockWithBusyNode1() throws Exception { + doTestMissingStripedBlock(2, 1); } @Test - public void testMissingStripedBlock() throws Exception { - final int numBlocks = 4; - DFSTestUtil.createStripedFile(cluster, filePath, - dirPath, numBlocks, 1, true); + public void testMissingStripedBlockWithBusyNode2() throws Exception { + doTestMissingStripedBlock(3, 1); + } - // make sure the file is complete in NN - final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() - .getINode4Write(filePath.toString()).asFile(); - assertFalse(fileNode.isUnderConstruction()); - assertTrue(fileNode.isStriped()); - BlockInfo[] blocks = fileNode.getBlocks(); - assertEquals(numBlocks, blocks.length); - for (BlockInfo blk : blocks) { - assertTrue(blk.isStriped()); - assertTrue(blk.isComplete()); - assertEquals(BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes()); - final BlockInfoStriped sb = (BlockInfoStriped) blk; - assertEquals(GROUP_SIZE, sb.numNodes()); - } + /** + * Start GROUP_SIZE + 1 datanodes. + * Inject striped blocks to first GROUP_SIZE datanodes. + * Then make numOfBusy datanodes busy, make numOfMissed datanodes missed. + * Then trigger BlockManager to compute recovery works. (so all recovery work + * will be scheduled to the last datanode) + * Finally, verify the recovery work of the last datanode. + */ + private void doTestMissingStripedBlock(int numOfMissed, int numOfBusy) + throws Exception { + Configuration conf = new HdfsConfiguration(); + initConf(conf); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE + 1) + .build(); + + try { + cluster.waitActive(); + final int numBlocks = 4; + DFSTestUtil.createStripedFile(cluster, filePath, + dirPath, numBlocks, 1, true); + // all blocks will be located at first GROUP_SIZE DNs, the last DN is + // empty because of the util function createStripedFile - final BlockManager bm = cluster.getNamesystem().getBlockManager(); - BlockInfo firstBlock = fileNode.getBlocks()[0]; - DatanodeStorageInfo[] storageInfos = bm.getStorages(firstBlock); + // make sure the file is complete in NN + final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() + .getINode4Write(filePath.toString()).asFile(); + assertFalse(fileNode.isUnderConstruction()); + assertTrue(fileNode.isStriped()); + BlockInfo[] blocks = fileNode.getBlocks(); + assertEquals(numBlocks, blocks.length); + for (BlockInfo blk : blocks) { + assertTrue(blk.isStriped()); + assertTrue(blk.isComplete()); + assertEquals(BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, + blk.getNumBytes()); + final BlockInfoStriped sb = (BlockInfoStriped) blk; + assertEquals(GROUP_SIZE, sb.numNodes()); + } - DatanodeDescriptor secondDn = storageInfos[1].getDatanodeDescriptor(); - assertEquals(numBlocks, secondDn.numBlocks()); + final BlockManager bm = cluster.getNamesystem().getBlockManager(); + BlockInfo firstBlock = fileNode.getBlocks()[0]; + DatanodeStorageInfo[] storageInfos = bm.getStorages(firstBlock); - bm.getDatanodeManager().removeDatanode(secondDn); + // make numOfBusy nodes busy + int i = 0; + for (; i < numOfBusy; i++) { + DatanodeDescriptor busyNode = storageInfos[i].getDatanodeDescriptor(); + for (int j = 0; j < maxReplicationStreams + 1; j++) { + BlockManagerTestUtil.addBlockToBeReplicated(busyNode, new Block(j), + new DatanodeStorageInfo[]{storageInfos[0]}); + } + } - BlockManagerTestUtil.getComputedDatanodeWork(bm); + // make numOfMissed internal blocks missed + for (; i < numOfBusy + numOfMissed; i++) { + DatanodeDescriptor missedNode = storageInfos[i].getDatanodeDescriptor(); + assertEquals(numBlocks, missedNode.numBlocks()); + bm.getDatanodeManager().removeDatanode(missedNode); + } - // all the recovery work will be scheduled on the last DN - DataNode lastDn = cluster.getDataNodes().get(GROUP_SIZE); - DatanodeDescriptor last = + BlockManagerTestUtil.getComputedDatanodeWork(bm); + + // all the recovery work will be scheduled on the last DN + DataNode lastDn = cluster.getDataNodes().get(GROUP_SIZE); + DatanodeDescriptor last = bm.getDatanodeManager().getDatanode(lastDn.getDatanodeId()); - assertEquals("Counting the number of outstanding EC tasks", numBlocks, - last.getNumberOfBlocksToBeErasureCoded()); - List recovery = last.getErasureCodeCommand(numBlocks); - for (BlockECRecoveryInfo info : recovery) { - assertEquals(1, info.getTargetDnInfos().length); - assertEquals(last, info.getTargetDnInfos()[0]); - assertEquals(GROUP_SIZE - 1, info.getSourceDnInfos().length); - assertEquals(GROUP_SIZE - 1, info.getLiveBlockIndices().length); + assertEquals("Counting the number of outstanding EC tasks", numBlocks, + last.getNumberOfBlocksToBeErasureCoded()); + List recovery = + last.getErasureCodeCommand(numBlocks); + for (BlockECRecoveryInfo info : recovery) { + assertEquals(1, info.getTargetDnInfos().length); + assertEquals(last, info.getTargetDnInfos()[0]); + assertEquals(info.getSourceDnInfos().length, + info.getLiveBlockIndices().length); + if (GROUP_SIZE - numOfMissed == NUM_DATA_BLOCKS) { + // It's a QUEUE_HIGHEST_PRIORITY block, so the busy DNs will be chosen + // to make sure we have NUM_DATA_BLOCKS DNs to do recovery work. + assertEquals(NUM_DATA_BLOCKS, info.getSourceDnInfos().length); + } else { + // The block has no highest priority, so we don't use the busy DNs as + // sources + assertEquals(GROUP_SIZE - numOfMissed - numOfBusy, + info.getSourceDnInfos().length); + } + } + } finally { + cluster.shutdown(); } } } From 42a1d0dbde2582867669268a3f7ad2476dbf629e Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 8 Jul 2015 22:52:09 -0700 Subject: [PATCH 170/212] Addendum fix for HDFS-8323 (adding back a mistakenly deleted line). --- .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 2 ++ 1 file changed, 2 insertions(+) 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 a238eeeefbaad..50ad2517807d7 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 @@ -5827,6 +5827,8 @@ LocatedBlock bumpBlockGenerationStamp(ExtendedBlock block, locatedBlock = BlockManager.newLocatedBlock( block, file.getLastBlock(), null, -1); + blockManager.setBlockToken(locatedBlock, + BlockTokenIdentifier.AccessMode.WRITE); } finally { writeUnlock(); } From 48f3830f2171420d65e74d7a44a6e7905012fe0d Mon Sep 17 00:00:00 2001 From: Walter Su Date: Thu, 9 Jul 2015 16:18:05 +0800 Subject: [PATCH 171/212] HDFS-8719. Erasure Coding: client generates too many small packets when writing parity data. Contributed by Li Bo. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 2 +- .../java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 58b91b60b747d..cdbef055ccc46 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -332,3 +332,6 @@ HDFS-8461. Erasure coding: fix priority level of UnderReplicatedBlocks for striped block. (Walter Su via jing9) + + HDFS-8719. Erasure Coding: client generates too many small packets when + writing parity data. (Li Bo via waltersu4549) 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 ea1ea269da581..9e201ad5d921a 100755 --- 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 @@ -459,7 +459,7 @@ void setCurrentPacket2Empty() throws InterruptedIOException { * write filled up its partial chunk. Tell the summer to generate full * crc chunks from now on. */ - private void adjustChunkBoundary() { + protected void adjustChunkBoundary() { if (streamer.getAppendChunk() && streamer.getBytesCurBlock() % bytesPerChecksum == 0) { streamer.setAppendChunk(false); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index cd7f6d411020c..4234351d36906 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -318,6 +318,7 @@ private synchronized StripedDataStreamer getCurrentStreamer() { private synchronized StripedDataStreamer setCurrentStreamer(int i) { streamer = streamers.get(i); + adjustChunkBoundary(); return getCurrentStreamer(); } From 2470a7bf88c7d7e0889905e35671c440c3ae649b Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 9 Jul 2015 10:55:28 -0700 Subject: [PATCH 172/212] HDFS-8563. Erasure Coding: fsck handles file smaller than a full stripe. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hdfs/server/namenode/NamenodeFsck.java | 59 +++++++------------ .../hadoop/hdfs/server/namenode/TestFsck.java | 18 ++++-- 3 files changed, 36 insertions(+), 44 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index cdbef055ccc46..7db7f1534396a 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -335,3 +335,6 @@ HDFS-8719. Erasure Coding: client generates too many small packets when writing parity data. (Li Bo via waltersu4549) + + HDFS-8563. Erasure Coding: fsck handles file smaller than a full stripe. + (Walter Su via jing9) 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 0737502ea5380..3e4a287893e72 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 @@ -65,6 +65,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus; @@ -74,7 +75,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.NodeBase; @@ -247,7 +247,7 @@ public void blockIdCK(String blockId) { //get blockInfo Block block = new Block(Block.getBlockId(blockId)); //find which file this block belongs to - BlockInfo blockInfo = namenode.getNamesystem().getStoredBlock(block); + BlockInfo blockInfo = bm.getStoredBlock(block); if(blockInfo == null) { out.println("Block "+ blockId +" " + NONEXISTENT_STATUS); LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS); @@ -556,6 +556,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res final BlockInfo storedBlock = bm.getStoredBlock( block.getLocalBlock()); + final int minReplication = bm.getMinStorageNum(storedBlock); // count decommissionedReplicas / decommissioningReplicas NumberReplicas numberReplicas = bm.countNodes(storedBlock); int decommissionedReplicas = numberReplicas.decommissioned(); @@ -571,26 +572,17 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res // count expected replicas short targetFileReplication; - if(file.getReplication() == 0) { - final FSNamesystem fsn = namenode.getNamesystem(); - final ECSchema ecSchema; - fsn.readLock(); - try { - INode inode = namenode.getNamesystem().getFSDirectory() - .getINode(path); - INodesInPath iip = INodesInPath.fromINode(inode); - ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(fsn, iip); - } finally { - fsn.readUnlock(); - } - targetFileReplication = (short) (ecSchema.getNumDataUnits() + ecSchema.getNumParityUnits()); + if (file.getECSchema() != null) { + assert storedBlock instanceof BlockInfoStriped; + targetFileReplication = ((BlockInfoStriped) storedBlock) + .getRealTotalBlockNum(); } else { targetFileReplication = file.getReplication(); } res.numExpectedReplicas += targetFileReplication; // count under min repl'd blocks - if(totalReplicasPerBlock < res.minReplication){ + if(totalReplicasPerBlock < minReplication){ res.numUnderMinReplicatedBlocks++; } @@ -611,7 +603,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res } // count minimally replicated blocks - if (totalReplicasPerBlock >= res.minReplication) + if (totalReplicasPerBlock >= minReplication) res.numMinReplicatedBlocks++; // count missing replicas / under replicated blocks @@ -1026,12 +1018,6 @@ static class Result { long totalOpenFilesSize = 0L; long totalReplicas = 0L; - final int minReplication; - - Result(int minReplication) { - this.minReplication = minReplication; - } - /** * DFS is considered healthy if there are no missing blocks. */ @@ -1062,12 +1048,13 @@ float getReplicationFactor() { @VisibleForTesting static class ReplicationResult extends Result { final short replication; + final short minReplication; ReplicationResult(Configuration conf) { - super(conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, - DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT)); this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT); + this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT); } @Override @@ -1171,15 +1158,11 @@ public String toString() { @VisibleForTesting static class ErasureCodingResult extends Result { - final String ecSchema; + final String defaultSchema; ErasureCodingResult(Configuration conf) { - this(ErasureCodingSchemaManager.getSystemDefaultSchema()); - } - - ErasureCodingResult(ECSchema ecSchema) { - super(ecSchema.getNumDataUnits()); - this.ecSchema = ecSchema.getSchemaName(); + defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema() + .getSchemaName(); } @Override @@ -1213,8 +1196,6 @@ public String toString() { ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks)) .append(" %)"); } - res.append("\n ").append("MIN REQUIRED EC BLOCK:\t") - .append(minReplication); } if(corruptFiles>0) { res.append( @@ -1251,18 +1232,18 @@ public String toString() { ((float) (numUnderReplicatedBlocks * 100) / (float) totalBlocks)) .append(" %)"); } - res.append("\n Unsatisfactory placement block groups:\t\t") + res.append("\n Unsatisfactory placement block groups:\t") .append(numMisReplicatedBlocks); if (totalBlocks > 0) { res.append(" (").append( ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks)) .append(" %)"); } - res.append("\n Default schema:\t").append(ecSchema) + res.append("\n Default schema:\t\t").append(defaultSchema) .append("\n Average block group size:\t").append( getReplicationFactor()).append("\n Missing block groups:\t\t").append( missingIds.size()).append("\n Corrupt block groups:\t\t").append( - corruptBlocks).append("\n Missing ec-blocks:\t\t").append( + corruptBlocks).append("\n Missing internal blocks:\t").append( missingReplicas); if (totalReplicas > 0) { res.append(" (").append( @@ -1270,11 +1251,11 @@ public String toString() { " %)"); } if (decommissionedReplicas > 0) { - res.append("\n Decommissioned ec-blocks:\t").append( + res.append("\n Decommissioned internal blocks:\t").append( decommissionedReplicas); } if (decommissioningReplicas > 0) { - res.append("\n Decommissioning ec-blocks:\t").append( + res.append("\n Decommissioning internal blocks:\t").append( decommissioningReplicas); } return res.toString(); 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 e5482d414527d..7e2b41d4dab0a 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 @@ -1648,23 +1648,31 @@ public void testECFsck() throws Exception { + ErasureCodingSchemaManager.getSystemDefaultSchema().getNumParityUnits(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build(); fs = cluster.getFileSystem(); + + // create a contiguous file Path replDirPath = new Path("/replicated"); Path replFilePath = new Path(replDirPath, "replfile"); final short factor = 3; DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0); DFSTestUtil.waitReplication(fs, replFilePath, factor); + + // create a large striped file Path ecDirPath = new Path("/striped"); - Path ecFilePath = new Path(ecDirPath, "ecfile"); - final int numBlocks = 4; - DFSTestUtil.createStripedFile(cluster, ecFilePath, ecDirPath, numBlocks, 2, true); + Path largeFilePath = new Path(ecDirPath, "largeFile"); + DFSTestUtil.createStripedFile(cluster, largeFilePath, ecDirPath, 1, 2, true); + + // create a small striped file + Path smallFilePath = new Path(ecDirPath, "smallFile"); + DFSTestUtil.writeFile(fs, smallFilePath, "hello world!"); + long replTime = fs.getFileStatus(replFilePath).getAccessTime(); - long ecTime = fs.getFileStatus(ecFilePath).getAccessTime(); + long ecTime = fs.getFileStatus(largeFilePath).getAccessTime(); Thread.sleep(precision); setupAuditLogs(); String outStr = runFsck(conf, 0, true, "/"); verifyAuditLogs(); assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime()); - assertEquals(ecTime, fs.getFileStatus(ecFilePath).getAccessTime()); + assertEquals(ecTime, fs.getFileStatus(largeFilePath).getAccessTime()); System.out.println(outStr); assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS)); if (fs != null) {try{fs.close();} catch(Exception e){}} From e692c7dd92c0d0fef50ce08edd972000151a9fe4 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 9 Jul 2015 14:07:37 -0700 Subject: [PATCH 173/212] HDFS-8484. Erasure coding: Two contiguous blocks occupy IDs belong to same striped group. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../server/blockmanagement/BlockManager.java | 20 ++++++++++--------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 7db7f1534396a..90f6732da3aea 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -338,3 +338,6 @@ HDFS-8563. Erasure Coding: fsck handles file smaller than a full stripe. (Walter Su via jing9) + + HDFS-8484. Erasure coding: Two contiguous blocks occupy IDs belong to same + striped group. (Walter Su via jing9) 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 c98bdb04649f7..968dc0c9d4288 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 @@ -3690,17 +3690,19 @@ public void removeBlock(BlockInfo block) { } public BlockInfo getStoredBlock(Block block) { - BlockInfo info = null; - if (BlockIdManager.isStripedBlockID(block.getBlockId())) { - info = blocksMap.getStoredBlock( + if (!BlockIdManager.isStripedBlockID(block.getBlockId())) { + return blocksMap.getStoredBlock(block); + } + if (!hasNonEcBlockUsingStripedID) { + return blocksMap.getStoredBlock( new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); - if ((info == null) && hasNonEcBlockUsingStripedID){ - info = blocksMap.getStoredBlock(block); - } - } else { - info = blocksMap.getStoredBlock(block); } - return info; + BlockInfo info = blocksMap.getStoredBlock(block); + if (info != null) { + return info; + } + return blocksMap.getStoredBlock( + new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); } /** updates a block in under replication queue */ From f4098dfae44b6dd287a3054f2c04658773b4f466 Mon Sep 17 00:00:00 2001 From: boli2 Date: Mon, 13 Jul 2015 00:41:36 -0400 Subject: [PATCH 174/212] HDFS-8744. Erasure Coding: the number of chunks in packet is not updated when writing parity data. Contributed by Li Bo --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 2 +- .../src/main/java/org/apache/hadoop/hdfs/DFSPacket.java | 6 +++--- .../java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java | 4 +++- 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 90f6732da3aea..7b6d1651953c7 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -341,3 +341,6 @@ HDFS-8484. Erasure coding: Two contiguous blocks occupy IDs belong to same striped group. (Walter Su via jing9) + + HDFS-8744. Erasure Coding: the number of chunks in packet is not updated + when writing parity data. (Li Bo) 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 9e201ad5d921a..f41044b299ef7 100755 --- 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 @@ -419,7 +419,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, currentPacket.writeChecksum(checksum, ckoff, cklen); currentPacket.writeData(b, offset, len); - currentPacket.incNumChunks(); + currentPacket.incNumChunks(1); streamer.incBytesCurBlock(len); // If packet is full, enqueue it for transmission diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java index a26e35e7c586b..2698de39c268f 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java @@ -259,10 +259,10 @@ synchronized int getNumChunks() { } /** - * increase the number of chunks by one + * increase the number of chunks by n */ - synchronized void incNumChunks() { - numChunks++; + synchronized void incNumChunks(int n) { + numChunks += n; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 4234351d36906..e6de714abafc4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -389,11 +389,13 @@ private List generatePackets( int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum; int toWrite = byteBuffer.remaining() > maxBytesToPacket ? maxBytesToPacket: byteBuffer.remaining(); - int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * getChecksumSize(); + int chunks = (toWrite - 1) / bytesPerChecksum + 1; + int ckLen = chunks * getChecksumSize(); p.writeChecksum(checksumBuf, ckOff, ckLen); ckOff += ckLen; p.writeData(byteBuffer, toWrite); getCurrentStreamer().incBytesCurBlock(toWrite); + p.incNumChunks(chunks); packets.add(p); } return packets; From b1e6429a6be382c93e894e46f63037819a5608a6 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 13 Jul 2015 11:41:18 -0700 Subject: [PATCH 175/212] HDFS-8669. Erasure Coding: handle missing internal block locations in DFSStripedInputStream. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../org/apache/hadoop/hdfs/BlockReader.java | 4 +- .../hadoop/hdfs/DFSStripedInputStream.java | 331 +++++++++++------- .../hadoop/hdfs/util/StripedBlockUtil.java | 11 +- .../hadoop/hdfs/StripedFileTestUtil.java | 128 +++++++ .../TestReadStripedFileWithMissingBlocks.java | 150 ++++++++ .../hadoop/hdfs/TestWriteReadStripedFile.java | 162 +-------- 7 files changed, 511 insertions(+), 278 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 7b6d1651953c7..cd9e19d8478f3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -344,3 +344,6 @@ HDFS-8744. Erasure Coding: the number of chunks in packet is not updated when writing parity data. (Li Bo) + + HDFS-8669. Erasure Coding: handle missing internal block locations in + DFSStripedInputStream. (jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java index 0a5511e16f0da..8f988afaa4650 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs; +import java.io.Closeable; import java.io.IOException; import java.util.EnumSet; @@ -31,7 +32,7 @@ * from a single datanode. */ @InterfaceAudience.Private -public interface BlockReader extends ByteBufferReadable { +public interface BlockReader extends ByteBufferReadable, Closeable { /* same interface as inputStream java.io.InputStream#read() @@ -63,6 +64,7 @@ public interface BlockReader extends ByteBufferReadable { * * @throws IOException */ + @Override // java.io.Closeable void close() throws IOException; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index a71da931dae33..75090036f551a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -43,6 +43,7 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -113,16 +114,43 @@ boolean include(long pos) { } } - private final BlockReader[] blockReaders; - /** - * when initializing block readers, their starting offsets are set to the same - * number: the smallest internal block offsets among all the readers. This is - * because it is possible that for some internal blocks we have to read - * "backwards" for decoding purpose. We thus use this offset array to track - * offsets for all the block readers so that we can skip data if necessary. - */ - private final long[] blockReaderOffsets; - private final DatanodeInfo[] currentNodes; + private static class BlockReaderInfo { + final BlockReader reader; + final DatanodeInfo datanode; + /** + * when initializing block readers, their starting offsets are set to the same + * number: the smallest internal block offsets among all the readers. This is + * because it is possible that for some internal blocks we have to read + * "backwards" for decoding purpose. We thus use this offset array to track + * offsets for all the block readers so that we can skip data if necessary. + */ + long blockReaderOffset; + LocatedBlock targetBlock; + /** + * We use this field to indicate whether we should use this reader. In case + * we hit any issue with this reader, we set this field to true and avoid + * using it for the next stripe. + */ + boolean shouldSkip = false; + + BlockReaderInfo(BlockReader reader, LocatedBlock targetBlock, + DatanodeInfo dn, long offset) { + this.reader = reader; + this.targetBlock = targetBlock; + this.datanode = dn; + this.blockReaderOffset = offset; + } + + void setOffset(long offset) { + this.blockReaderOffset = offset; + } + + void skip() { + this.shouldSkip = true; + } + } + + private final BlockReaderInfo[] blockReaders; private final int cellSize; private final short dataBlkNum; private final short parityBlkNum; @@ -151,9 +179,7 @@ boolean include(long pos) { dataBlkNum = (short) schema.getNumDataUnits(); parityBlkNum = (short) schema.getNumParityUnits(); groupSize = dataBlkNum + parityBlkNum; - blockReaders = new BlockReader[groupSize]; - blockReaderOffsets = new long[groupSize]; - currentNodes = new DatanodeInfo[groupSize]; + blockReaders = new BlockReaderInfo[groupSize]; curStripeRange = new StripeRange(0, 0); readingService = new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); @@ -218,18 +244,26 @@ private synchronized void blockSeekTo(long target) throws IOException { for (int i = 0; i < dataBlkNum; i++) { LocatedBlock targetBlock = targetBlocks[i]; if (targetBlock != null) { - DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); - if (retval != null) { - currentNodes[i] = retval.info; - blockReaders[i] = getBlockReaderWithRetry(targetBlock, + DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null); + if (dnInfo != null) { + BlockReader reader = getBlockReaderWithRetry(targetBlock, minOffset, targetBlock.getBlockSize() - minOffset, - retval.addr, retval.storageType, retval.info, target, retry); - blockReaderOffsets[i] = minOffset; + dnInfo.addr, dnInfo.storageType, dnInfo.info, target, retry); + if (reader != null) { + blockReaders[i] = new BlockReaderInfo(reader, targetBlock, + dnInfo.info, minOffset); + } } } } } + /** + * @throws IOException only when failing to refetch block token, which happens + * when this client cannot get located block information from NameNode. This + * method returns null instead of throwing exception when failing to connect + * to the DataNode. + */ private BlockReader getBlockReaderWithRetry(LocatedBlock targetBlock, long offsetInBlock, long length, InetSocketAddress targetAddr, StorageType storageType, DatanodeInfo datanode, long offsetInFile, @@ -275,21 +309,16 @@ protected void closeCurrentBlockReaders() { } for (int i = 0; i < groupSize; i++) { closeReader(i); - currentNodes[i] = null; + blockReaders[i] = null; } blockEnd = -1; } private void closeReader(int index) { if (blockReaders[index] != null) { - try { - blockReaders[index].close(); - } catch (IOException e) { - DFSClient.LOG.error("error closing blockReader " + index, e); - } - blockReaders[index] = null; + IOUtils.cleanup(DFSClient.LOG, blockReaders[index].reader); + blockReaders[index].skip(); } - blockReaderOffsets[index] = 0; } private long getOffsetInBlockGroup() { @@ -323,16 +352,14 @@ private void readOneStripe( AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(schema, cellSize, blockGroup, offsetInBlockGroup, offsetInBlockGroup + curStripeRange.length - 1, curStripeBuf); - // TODO handle null elements in blks (e.g., NN does not know locations for - // all the internal blocks) final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( blockGroup, cellSize, dataBlkNum, parityBlkNum); // read the whole stripe for (AlignedStripe stripe : stripes) { // Parse group to get chosen DN location StripeReader sreader = new StatefulStripeReader(readingService, stripe, - blks); - sreader.readStripe(blks, corruptedBlockMap); + blks, corruptedBlockMap); + sreader.readStripe(); } curStripeBuf.position(stripeBufOffset); curStripeBuf.limit(stripeLimit); @@ -549,14 +576,13 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, blockGroup, start, end, buf, offset); CompletionService readService = new ExecutorCompletionService<>( dfsClient.getStripedReadsThreadPool()); - // TODO handle null elements in blks (e.g., NN does not know locations for - // all the internal blocks) final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( blockGroup, cellSize, dataBlkNum, parityBlkNum); for (AlignedStripe stripe : stripes) { // Parse group to get chosen DN location - StripeReader preader = new PositionStripeReader(readService, stripe); - preader.readStripe(blks, corruptedBlockMap); + StripeReader preader = new PositionStripeReader(readService, stripe, + blks, corruptedBlockMap); + preader.readStripe(); } } @@ -586,43 +612,89 @@ private abstract class StripeReader { final Map, Integer> futures = new HashMap<>(); final AlignedStripe alignedStripe; final CompletionService service; + final LocatedBlock[] targetBlocks; + final Map> corruptedBlockMap; - StripeReader(CompletionService service, AlignedStripe alignedStripe) { + StripeReader(CompletionService service, AlignedStripe alignedStripe, + LocatedBlock[] targetBlocks, + Map> corruptedBlockMap) { this.service = service; this.alignedStripe = alignedStripe; + this.targetBlocks = targetBlocks; + this.corruptedBlockMap = corruptedBlockMap; } - /** submit reading chunk task */ - abstract void readChunk(final CompletionService service, - final LocatedBlock block, int chunkIndex, - Map> corruptedBlockMap); + abstract boolean readChunk(final CompletionService service, + final LocatedBlock block, int chunkIndex); - /** - * When seeing first missing block, initialize decode input buffers. - * Also prepare the reading for data blocks outside of the reading range. - */ - abstract void prepareDecodeInputs() throws IOException; + /** prepare all the data chunks */ + abstract void prepareDecodeInputs(); - /** - * Prepare reading for one more parity chunk. - */ - abstract void prepareParityChunk() throws IOException; + /** prepare the parity chunk and block reader if necessary */ + abstract boolean prepareParityChunk(int index) throws IOException; abstract void decode(); abstract void updateState4SuccessRead(StripingChunkReadResult result); + private void checkMissingBlocks() throws IOException { + if (alignedStripe.missingChunksNum > parityBlkNum) { + clearFutures(futures.keySet()); + throw new IOException(alignedStripe.missingChunksNum + + " missing blocks, the stripe is: " + alignedStripe); + } + } + + /** + * We need decoding. Thus go through all the data chunks and make sure we + * submit read requests for all of them. + */ + private void readDataForDecoding() throws IOException { + prepareDecodeInputs(); + for (int i = 0; i < dataBlkNum; i++) { + Preconditions.checkNotNull(alignedStripe.chunks[i]); + if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) { + if (!readChunk(service, targetBlocks[i], i)) { + alignedStripe.missingChunksNum++; + } + } + } + checkMissingBlocks(); + } + + void readParityChunks(int num) throws IOException { + for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num; + i++) { + if (alignedStripe.chunks[i] == null) { + if (prepareParityChunk(i) && readChunk(service, targetBlocks[i], i)) { + j++; + } else { + alignedStripe.missingChunksNum++; + } + } + } + checkMissingBlocks(); + } + /** read the whole stripe. do decoding if necessary */ - void readStripe(LocatedBlock[] blocks, - Map> corruptedBlockMap) - throws IOException { - assert alignedStripe.getSpanInBlock() > 0; - for (short i = 0; i < dataBlkNum; i++) { - if (alignedStripe.chunks[i] != null - && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { - readChunk(service, blocks[i], i, corruptedBlockMap); + void readStripe() throws IOException { + for (int i = 0; i < dataBlkNum; i++) { + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { + if (!readChunk(service, targetBlocks[i], i)) { + alignedStripe.missingChunksNum++; + } } } + // There are missing block locations at this stage. Thus we need to read + // the full stripe and one more parity block. + if (alignedStripe.missingChunksNum > 0) { + checkMissingBlocks(); + readDataForDecoding(); + // read parity chunks + readParityChunks(alignedStripe.missingChunksNum); + } + // TODO: for a full stripe we can start reading (dataBlkNum + 1) chunks // Input buffers for potential decode operation, which remains null until // first read failure @@ -648,24 +720,15 @@ void readStripe(LocatedBlock[] blocks, } } else { returnedChunk.state = StripingChunk.MISSING; - alignedStripe.missingChunksNum++; - if (alignedStripe.missingChunksNum > parityBlkNum) { - clearFutures(futures.keySet()); - throw new IOException("Too many blocks are missing: " - + alignedStripe); - } - - prepareDecodeInputs(); - prepareParityChunk(); // close the corresponding reader closeReader(r.index); - for (int i = 0; i < alignedStripe.chunks.length; i++) { - StripingChunk chunk = alignedStripe.chunks[i]; - if (chunk != null && chunk.state == StripingChunk.REQUESTED) { - readChunk(service, blocks[i], i, corruptedBlockMap); - } - } + final int missing = alignedStripe.missingChunksNum; + alignedStripe.missingChunksNum++; + checkMissingBlocks(); + + readDataForDecoding(); + readParityChunks(alignedStripe.missingChunksNum - missing); } } catch (InterruptedException ie) { String err = "Read request interrupted"; @@ -686,20 +749,24 @@ class PositionStripeReader extends StripeReader { private byte[][] decodeInputs = null; PositionStripeReader(CompletionService service, - AlignedStripe alignedStripe) { - super(service, alignedStripe); + AlignedStripe alignedStripe, LocatedBlock[] targetBlocks, + Map> corruptedBlockMap) { + super(service, alignedStripe, targetBlocks, corruptedBlockMap); } @Override - void readChunk(final CompletionService service, - final LocatedBlock block, int chunkIndex, - Map> corruptedBlockMap) { + boolean readChunk(final CompletionService service, + final LocatedBlock block, int chunkIndex) { + final StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + if (block == null) { + chunk.state = StripingChunk.MISSING; + return false; + } DatanodeInfo loc = block.getLocations()[0]; StorageType type = block.getStorageTypes()[0]; DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), type); - StripingChunk chunk = alignedStripe.chunks[chunkIndex]; chunk.state = StripingChunk.PENDING; Callable readCallable = getFromOneDataNode(dnAddr, block, alignedStripe.getOffsetInBlock(), @@ -715,6 +782,7 @@ void readChunk(final CompletionService service, + alignedStripe.getSpanInBlock() - 1)); } futures.put(getFromDNRequest, chunkIndex); + return true; } @Override @@ -728,18 +796,15 @@ void prepareDecodeInputs() { } @Override - void prepareParityChunk() { - for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { - if (alignedStripe.chunks[i] == null) { - final int decodeIndex = convertIndex4Decode(i, - dataBlkNum, parityBlkNum); - alignedStripe.chunks[i] = - new StripingChunk(decodeInputs[decodeIndex]); - alignedStripe.chunks[i].addByteArraySlice(0, - (int) alignedStripe.getSpanInBlock()); - break; - } - } + boolean prepareParityChunk(int index) { + Preconditions.checkState(index >= dataBlkNum && + alignedStripe.chunks[index] == null); + final int decodeIndex = convertIndex4Decode(index, dataBlkNum, + parityBlkNum); + alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); + alignedStripe.chunks[index].addByteArraySlice(0, + (int) alignedStripe.getSpanInBlock()); + return true; } @Override @@ -753,39 +818,43 @@ void decode() { class StatefulStripeReader extends StripeReader { ByteBuffer[] decodeInputs; - final LocatedBlock[] targetBlocks; StatefulStripeReader(CompletionService service, - AlignedStripe alignedStripe, LocatedBlock[] targetBlocks) { - super(service, alignedStripe); - this.targetBlocks = targetBlocks; + AlignedStripe alignedStripe, LocatedBlock[] targetBlocks, + Map> corruptedBlockMap) { + super(service, alignedStripe, targetBlocks, corruptedBlockMap); } @Override - void readChunk(final CompletionService service, - final LocatedBlock block, int chunkIndex, Map> corruptedBlockMap) { - StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + boolean readChunk(final CompletionService service, + final LocatedBlock block, int chunkIndex) { + final StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + final BlockReaderInfo readerInfo = blockReaders[chunkIndex]; + if (readerInfo == null || block == null || readerInfo.shouldSkip) { + chunk.state = StripingChunk.MISSING; + return false; + } chunk.state = StripingChunk.PENDING; ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer); - Callable readCallable = readCell(blockReaders[chunkIndex], - currentNodes[chunkIndex], blockReaderOffsets[chunkIndex], + Callable readCallable = readCell(readerInfo.reader, + readerInfo.datanode, readerInfo.blockReaderOffset, alignedStripe.getOffsetInBlock(), strategy, chunk.byteBuffer.remaining(), block.getBlock(), corruptedBlockMap); Future request = readingService.submit(readCallable); futures.put(request, chunkIndex); + return true; } @Override void updateState4SuccessRead(StripingChunkReadResult result) { Preconditions.checkArgument( result.state == StripingChunkReadResult.SUCCESSFUL); - blockReaderOffsets[result.index] = - alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock(); + blockReaders[result.index].setOffset(alignedStripe.getOffsetInBlock() + + alignedStripe.getSpanInBlock()); } @Override - void prepareDecodeInputs() throws IOException { + void prepareDecodeInputs() { if (decodeInputs == null) { decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum]; ByteBuffer cur = curStripeBuf.duplicate(); @@ -799,52 +868,58 @@ void prepareDecodeInputs() throws IOException { parityBlkNum); decodeInputs[decodeIndex] = cur.slice(); if (alignedStripe.chunks[i] == null) { - alignedStripe.chunks[i] = - new StripingChunk(decodeInputs[decodeIndex]); + alignedStripe.chunks[i] = new StripingChunk( + decodeInputs[decodeIndex]); } } } } @Override - void prepareParityChunk() throws IOException { - for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { - if (alignedStripe.chunks[i] == null) { - final int decodeIndex = convertIndex4Decode(i, dataBlkNum, - parityBlkNum); - decodeInputs[decodeIndex] = ByteBuffer.allocateDirect( - (int) alignedStripe.range.spanInBlock); - alignedStripe.chunks[i] = - new StripingChunk(decodeInputs[decodeIndex]); - if (blockReaders[i] == null) { - prepareParityBlockReader(i); - } - break; - } + boolean prepareParityChunk(int index) throws IOException { + Preconditions.checkState(index >= dataBlkNum + && alignedStripe.chunks[index] == null); + if (blockReaders[index] != null && blockReaders[index].shouldSkip) { + alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING); + // we have failed the block reader before + return false; } + final int decodeIndex = convertIndex4Decode(index, dataBlkNum, + parityBlkNum); + decodeInputs[decodeIndex] = ByteBuffer.allocateDirect( + (int) alignedStripe.range.spanInBlock); + alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); + if (blockReaders[index] == null && !prepareParityBlockReader(index)) { + alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING); + return false; + } + return true; } - private void prepareParityBlockReader(int i) throws IOException { + private boolean prepareParityBlockReader(int i) throws IOException { // prepare the block reader for the parity chunk LocatedBlock targetBlock = targetBlocks[i]; if (targetBlock != null) { final long offsetInBlock = alignedStripe.getOffsetInBlock(); - DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null); - if (retval != null) { - currentNodes[i] = retval.info; - blockReaders[i] = getBlockReaderWithRetry(targetBlock, + DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null); + if (dnInfo != null) { + BlockReader reader = getBlockReaderWithRetry(targetBlock, offsetInBlock, targetBlock.getBlockSize() - offsetInBlock, - retval.addr, retval.storageType, retval.info, + dnInfo.addr, dnInfo.storageType, dnInfo.info, DFSStripedInputStream.this.getPos(), retry); - blockReaderOffsets[i] = offsetInBlock; + if (reader != null) { + blockReaders[i] = new BlockReaderInfo(reader, targetBlock, + dnInfo.info, offsetInBlock); + return true; + } } } + return false; } @Override void decode() { - // TODO no copy for data chunks. this depends on HADOOP-12047 for some - // decoders to work + // TODO no copy for data chunks. this depends on HADOOP-12047 final int span = (int) alignedStripe.getSpanInBlock(); for (int i = 0; i < alignedStripe.chunks.length; i++) { final int decodeIndex = convertIndex4Decode(i, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 579434b67069a..6bd5e1f01030f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -83,6 +83,7 @@ public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum]; for (short i = 0; i < locatedBGSize; i++) { final int idx = bg.getBlockIndices()[i]; + // for now we do not use redundant replica of an internal block if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) { lbs[idx] = constructInternalBlock(bg, i, cellSize, dataBlkNum, idx); @@ -212,7 +213,9 @@ public static StripingChunkReadResult getNextCompletedStripedRead( return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT); } } catch (ExecutionException e) { - DFSClient.LOG.warn("ExecutionException " + e); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("ExecutionException " + e); + } return new StripingChunkReadResult(futures.remove(future), StripingChunkReadResult.FAILED); } catch (CancellationException e) { @@ -623,7 +626,7 @@ private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, cellSize, dataBlkNum, i); if (internalBlkLen <= s.getOffsetInBlock()) { Preconditions.checkState(s.chunks[i] == null); - s.chunks[i] = new StripingChunk(); // chunk state is set to ALLZERO + s.chunks[i] = new StripingChunk(StripingChunk.ALLZERO); } } } @@ -841,10 +844,10 @@ public StripingChunk(ByteBuffer buf) { this.byteBuffer = buf; } - public StripingChunk() { + public StripingChunk(int state) { this.byteArray = null; this.byteBuffer = null; - this.state = ALLZERO; + this.state = state; } public void addByteArraySlice(int offset, int length) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index 23697040db733..815a50d80a5a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -18,9 +18,16 @@ package org.apache.hadoop.hdfs; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.web.ByteRangeInputStream; +import org.junit.Assert; +import java.io.EOFException; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Random; public class StripedFileTestUtil { @@ -56,4 +63,125 @@ static byte getByte(long pos) { final int mod = 29; return (byte) (pos % mod + 1); } + + static void verifyLength(FileSystem fs, Path srcPath, int fileLength) + throws IOException { + FileStatus status = fs.getFileStatus(srcPath); + Assert.assertEquals("File length should be the same", fileLength, status.getLen()); + } + + static void verifyPread(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, byte[] buf) throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102, + cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102, + cellSize * dataBlocks, fileLength - 102, fileLength - 1}; + for (int startOffset : startOffsets) { + startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); + int remaining = fileLength - startOffset; + in.readFully(startOffset, buf, 0, remaining); + for (int i = 0; i < remaining; i++) { + Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + + "same", expected[startOffset + i], buf[i]); + } + } + } + } + + static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, byte[] buf) throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + final byte[] result = new byte[fileLength]; + int readLen = 0; + int ret; + while ((ret = in.read(buf, 0, buf.length)) >= 0) { + System.arraycopy(buf, 0, result, readLen, ret); + readLen += ret; + } + Assert.assertEquals("The length of file should be the same to write size", + fileLength, readLen); + Assert.assertArrayEquals(expected, result); + } + } + + static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, ByteBuffer buf) throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + ByteBuffer result = ByteBuffer.allocate(fileLength); + int readLen = 0; + int ret; + while ((ret = in.read(buf)) >= 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + Assert.assertEquals("The length of file should be the same to write size", + fileLength, readLen); + Assert.assertArrayEquals(expected, result.array()); + } + } + + static void verifySeek(FileSystem fs, Path srcPath, int fileLength) + throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + // seek to 1/2 of content + int pos = fileLength / 2; + assertSeekAndRead(in, pos, fileLength); + + // seek to 1/3 of content + pos = fileLength / 3; + assertSeekAndRead(in, pos, fileLength); + + // seek to 0 pos + pos = 0; + assertSeekAndRead(in, pos, fileLength); + + if (fileLength > cellSize) { + // seek to cellSize boundary + pos = cellSize - 1; + assertSeekAndRead(in, pos, fileLength); + } + + if (fileLength > cellSize * dataBlocks) { + // seek to striped cell group boundary + pos = cellSize * dataBlocks - 1; + assertSeekAndRead(in, pos, fileLength); + } + + if (fileLength > blockSize * dataBlocks) { + // seek to striped block group boundary + pos = blockSize * dataBlocks - 1; + assertSeekAndRead(in, pos, fileLength); + } + + if (!(in.getWrappedStream() instanceof ByteRangeInputStream)) { + try { + in.seek(-1); + Assert.fail("Should be failed if seek to negative offset"); + } catch (EOFException e) { + // expected + } + + try { + in.seek(fileLength + 1); + Assert.fail("Should be failed if seek after EOF"); + } catch (EOFException e) { + // expected + } + } + } + } + + static void assertSeekAndRead(FSDataInputStream fsdis, int pos, + int writeBytes) throws IOException { + fsdis.seek(pos); + byte[] buf = new byte[writeBytes]; + int readLen = StripedFileTestUtil.readAll(fsdis, buf); + Assert.assertEquals(readLen, writeBytes - pos); + for (int i = 0; i < readLen; i++) { + Assert.assertEquals("Byte at " + i + " should be the same", + StripedFileTestUtil.getByte(pos + i), buf[i]); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java new file mode 100644 index 0000000000000..4c2438d76bcc6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java @@ -0,0 +1,150 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; + +/** + * Test reading a striped file when some of its blocks are missing (not included + * in the block locations returned by the NameNode). + */ +public class TestReadStripedFileWithMissingBlocks { + public static final Log LOG = LogFactory + .getLog(TestReadStripedFileWithMissingBlocks.class); + private static MiniDFSCluster cluster; + private static FileSystem fs; + private static Configuration conf = new HdfsConfiguration(); + private final int fileLength = blockSize * dataBlocks + 123; + + @Before + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().createErasureCodingZone("/", + null, cellSize); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testReadFileWithMissingBlocks1() throws IOException { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 0); + } + + @Test + public void testReadFileWithMissingBlocks2() throws IOException { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 1); + } + + @Test + public void testReadFileWithMissingBlocks3() throws IOException { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 2); + } + + @Test + public void testReadFileWithMissingBlocks4() throws IOException { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 0); + } + + @Test + public void testReadFileWithMissingBlocks5() throws IOException { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 1); + } + + @Test + public void testReadFileWithMissingBlocks6() throws IOException { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 3, 0); + } + + private void readFileWithMissingBlocks(Path srcPath, int fileLength, + int missingDataNum, int missingParityNum) + throws IOException { + LOG.info("readFileWithMissingBlocks: (" + missingDataNum + "," + + missingParityNum + ")"); + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); + int dataBlocks = (fileLength - 1) / cellSize + 1; + BlockLocation[] locs = fs.getFileBlockLocations(srcPath, 0, cellSize); + + int[] missingDataNodes = new int[missingDataNum + missingParityNum]; + for (int i = 0; i < missingDataNum; i++) { + missingDataNodes[i] = i; + } + for (int i = 0; i < missingParityNum; i++) { + missingDataNodes[i + missingDataNum] = i + + Math.min(StripedFileTestUtil.dataBlocks, dataBlocks); + } + stopDataNodes(locs, missingDataNodes); + + // make sure there are missing block locations + BlockLocation[] newLocs = fs.getFileBlockLocations(srcPath, 0, cellSize); + Assert.assertTrue(newLocs[0].getNames().length < locs[0].getNames().length); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + smallBuf); + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + // delete the file + fs.delete(srcPath, true); + } + + private void stopDataNodes(BlockLocation[] locs, int[] datanodes) + throws IOException { + if (locs != null && locs.length > 0) { + for (int failedDNIdx : datanodes) { + String name = (locs[0].getNames())[failedDNIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + cluster.setDataNodeDead(dn.getDatanodeId()); + LOG.info("stop datanode " + failedDNIdx); + break; + } + } + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 272650d358c35..2f9322dcdd0c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -21,20 +21,15 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.web.ByteRangeInputStream; import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; @@ -48,11 +43,10 @@ public class TestWriteReadStripedFile { public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class); private static MiniDFSCluster cluster; private static FileSystem fs; - private static Configuration conf; + private static Configuration conf = new HdfsConfiguration(); @Before public void setup() throws IOException { - conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", @@ -175,18 +169,6 @@ public void testFileMoreThanABlockGroup3() throws IOException { + cellSize + 123, true); } - private void assertSeekAndRead(FSDataInputStream fsdis, int pos, - int writeBytes) throws IOException { - fsdis.seek(pos); - byte[] buf = new byte[writeBytes]; - int readLen = StripedFileTestUtil.readAll(fsdis, buf); - Assert.assertEquals(readLen, writeBytes - pos); - for (int i = 0; i < readLen; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", - StripedFileTestUtil.getByte(pos + i), buf[i]); - } - } - private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) throws IOException { testOneFileUsingDFSStripedInputStream(src, fileLength, false); @@ -198,7 +180,7 @@ private void testOneFileUsingDFSStripedInputStream(String src, int fileLength, Path srcPath = new Path(src); DFSTestUtil.writeFile(fs, srcPath, new String(expected)); - verifyLength(fs, srcPath, fileLength); + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); if (withDataNodeFailure) { int dnIndex = 1; // TODO: StripedFileTestUtil.random.nextInt(dataBlocks); @@ -208,14 +190,16 @@ private void testOneFileUsingDFSStripedInputStream(String src, int fileLength, byte[] smallBuf = new byte[1024]; byte[] largeBuf = new byte[fileLength + 100]; - verifyPread(fs, srcPath, fileLength, expected, largeBuf); + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); - verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); - verifySeek(fs, srcPath, fileLength); - verifyStatefulRead(fs, srcPath, fileLength, expected, + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + largeBuf); + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, ByteBuffer.allocate(fileLength + 100)); - verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); - verifyStatefulRead(fs, srcPath, fileLength, expected, + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + smallBuf); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, ByteBuffer.allocate(1024)); } @@ -241,130 +225,18 @@ public void testWriteReadUsingWebHdfs() throws Exception { final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME); - Path srcPath = new Path("/testWriteReadUsingWebHdfs_stripe"); + Path srcPath = new Path("/testWriteReadUsingWebHdfs"); DFSTestUtil.writeFile(fs, srcPath, new String(expected)); - verifyLength(fs, srcPath, fileLength); + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); byte[] smallBuf = new byte[1024]; byte[] largeBuf = new byte[fileLength + 100]; - verifyPread(fs, srcPath, fileLength, expected, largeBuf); - - verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); - verifySeek(fs, srcPath, fileLength); - verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); - //webhdfs doesn't support bytebuffer read - } - - void verifyLength(FileSystem fs, Path srcPath, int fileLength) - throws IOException { - FileStatus status = fs.getFileStatus(srcPath); - Assert.assertEquals("File length should be the same", - fileLength, status.getLen()); - } - - void verifyPread(FileSystem fs, Path srcPath, int fileLength, - byte[] expected, byte[] buf) throws IOException { - try (FSDataInputStream in = fs.open(srcPath)) { - int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102, - cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102, - cellSize * dataBlocks, fileLength - 102, fileLength - 1}; - for (int startOffset : startOffsets) { - startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); - int remaining = fileLength - startOffset; - in.readFully(startOffset, buf, 0, remaining); - for (int i = 0; i < remaining; i++) { - Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + - "same", expected[startOffset + i], buf[i]); - } - } - } - } - - void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, - byte[] expected, byte[] buf) throws IOException { - try (FSDataInputStream in = fs.open(srcPath)) { - final byte[] result = new byte[fileLength]; - int readLen = 0; - int ret; - while ((ret = in.read(buf, 0, buf.length)) >= 0) { - System.arraycopy(buf, 0, result, readLen, ret); - readLen += ret; - } - Assert.assertEquals("The length of file should be the same to write size", - fileLength, readLen); - Assert.assertArrayEquals(expected, result); - } - } - + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); - void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, - byte[] expected, ByteBuffer buf) throws IOException { - try (FSDataInputStream in = fs.open(srcPath)) { - ByteBuffer result = ByteBuffer.allocate(fileLength); - int readLen = 0; - int ret; - while ((ret = in.read(buf)) >= 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); - } - Assert.assertEquals("The length of file should be the same to write size", - fileLength, readLen); - Assert.assertArrayEquals(expected, result.array()); - } - } - - - void verifySeek(FileSystem fs, Path srcPath, int fileLength) - throws IOException { - try (FSDataInputStream in = fs.open(srcPath)) { - // seek to 1/2 of content - int pos = fileLength / 2; - assertSeekAndRead(in, pos, fileLength); - - // seek to 1/3 of content - pos = fileLength / 3; - assertSeekAndRead(in, pos, fileLength); - - // seek to 0 pos - pos = 0; - assertSeekAndRead(in, pos, fileLength); - - if (fileLength > cellSize) { - // seek to cellSize boundary - pos = cellSize - 1; - assertSeekAndRead(in, pos, fileLength); - } - - if (fileLength > cellSize * dataBlocks) { - // seek to striped cell group boundary - pos = cellSize * dataBlocks - 1; - assertSeekAndRead(in, pos, fileLength); - } - - if (fileLength > blockSize * dataBlocks) { - // seek to striped block group boundary - pos = blockSize * dataBlocks - 1; - assertSeekAndRead(in, pos, fileLength); - } - - if (!(in.getWrappedStream() instanceof ByteRangeInputStream)) { - try { - in.seek(-1); - Assert.fail("Should be failed if seek to negative offset"); - } catch (EOFException e) { - // expected - } - - try { - in.seek(fileLength + 1); - Assert.fail("Should be failed if seek after EOF"); - } catch (EOFException e) { - // expected - } - } - } + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); + // webhdfs doesn't support bytebuffer read } } From 6ff957be88d48a8b41e9fcbe4cf466d672cd7bc1 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 14 Jul 2015 10:55:58 -0700 Subject: [PATCH 176/212] HDFS-8702. Erasure coding: update BlockManager.blockHasEnoughRacks(..) logic for striped block. Contributed by Kai Sasaki. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../server/blockmanagement/BlockManager.java | 43 ++++++++++++++++++- 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index cd9e19d8478f3..2b9129598903f 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -347,3 +347,6 @@ HDFS-8669. Erasure Coding: handle missing internal block locations in DFSStripedInputStream. (jing9) + + HDFS-8702. Erasure coding: update BlockManager.blockHasEnoughRacks(..) logic + for striped block. (Kai Sasaki via jing9) 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 968dc0c9d4288..1aaf22569c4c1 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 @@ -3819,14 +3819,53 @@ private int invalidateWorkForOneNode(DatanodeInfo dn) { return toInvalidate.size(); } - // TODO: update the enough rack logic for striped blocks boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) { if (!this.shouldCheckForEnoughRacks) { return true; } - boolean enoughRacks = false; Collection corruptNodes = corruptReplicas.getNodes(storedBlock); + + if (storedBlock.isStriped()) { + return blockHasEnoughRacksStriped(storedBlock, corruptNodes); + } else { + return blockHashEnoughRacksContiguous(storedBlock, expectedStorageNum, + corruptNodes); + } + } + + /** + * Verify whether given striped block is distributed through enough racks. + * As dicussed in HDFS-7613, ec file requires racks at least as many as + * the number of data block number. + */ + boolean blockHasEnoughRacksStriped(BlockInfo storedBlock, + Collection corruptNodes) { + if (!datanodeManager.hasClusterEverBeenMultiRack()) { + return true; + } + boolean enoughRacks = false; + Set rackNameSet = new HashSet<>(); + int dataBlockNum = ((BlockInfoStriped)storedBlock).getRealDataBlockNum(); + for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) { + final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); + if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { + if ((corruptNodes == null) || !corruptNodes.contains(cur)) { + String rackNameNew = cur.getNetworkLocation(); + rackNameSet.add(rackNameNew); + if (rackNameSet.size() >= dataBlockNum) { + enoughRacks = true; + break; + } + } + } + } + return enoughRacks; + } + + boolean blockHashEnoughRacksContiguous(BlockInfo storedBlock, + int expectedStorageNum, Collection corruptNodes) { + boolean enoughRacks = false; String rackName = null; for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) { final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); From 0a93712f3b9b36d746577dca5da0f7f09756fcca Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 14 Jul 2015 13:10:51 -0700 Subject: [PATCH 177/212] HDFS-8734. Erasure Coding: fix one cell need two packets. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSOutputStream.java | 2 +- .../org/apache/hadoop/hdfs/DFSPacket.java | 6 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 85 ++++++++----------- 4 files changed, 41 insertions(+), 55 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2b9129598903f..93c3162f352ac 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -350,3 +350,6 @@ HDFS-8702. Erasure coding: update BlockManager.blockHasEnoughRacks(..) logic for striped block. (Kai Sasaki via jing9) + + HDFS-8734. Erasure Coding: fix one cell need two packets. (Walter Su via + jing9) 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 f41044b299ef7..9e201ad5d921a 100755 --- 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 @@ -419,7 +419,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, currentPacket.writeChecksum(checksum, ckoff, cklen); currentPacket.writeData(b, offset, len); - currentPacket.incNumChunks(1); + currentPacket.incNumChunks(); streamer.incBytesCurBlock(len); // If packet is full, enqueue it for transmission diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java index 2698de39c268f..a26e35e7c586b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java @@ -259,10 +259,10 @@ synchronized int getNumChunks() { } /** - * increase the number of chunks by n + * increase the number of chunks by one */ - synchronized void incNumChunks(int n) { - numChunks += n; + synchronized void incNumChunks() { + numChunks++; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index e6de714abafc4..a4bb49de3f62b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -254,6 +254,7 @@ private void flipDataBuffers() { private final CellBuffers cellBuffers; private final RawErasureEncoder encoder; private final List streamers; + private final DFSPacket[] currentPackets; // current Packet of each streamer /** Size of each striping cell, must be a multiple of bytesPerChecksum */ private final int cellSize; @@ -301,6 +302,7 @@ ExtendedBlock getBlock() { s.add(streamer); } streamers = Collections.unmodifiableList(s); + currentPackets = new DFSPacket[streamers.size()]; setCurrentStreamer(0); } @@ -316,9 +318,18 @@ private synchronized StripedDataStreamer getCurrentStreamer() { return (StripedDataStreamer)streamer; } - private synchronized StripedDataStreamer setCurrentStreamer(int i) { - streamer = streamers.get(i); + private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) + throws IOException { + // backup currentPacket for current streamer + int oldIdx = streamers.indexOf(streamer); + if (oldIdx >= 0) { + currentPackets[oldIdx] = currentPacket; + } + + streamer = streamers.get(newIdx); + currentPacket = currentPackets[newIdx]; adjustChunkBoundary(); + return getCurrentStreamer(); } @@ -366,41 +377,6 @@ private void handleStreamerFailure(String err, currentPacket = null; } - /** - * Generate packets from a given buffer. This is only used for streamers - * writing parity blocks. - * - * @param byteBuffer the given buffer to generate packets - * @param checksumBuf the checksum buffer - * @return packets generated - * @throws IOException - */ - private List generatePackets( - ByteBuffer byteBuffer, byte[] checksumBuf) throws IOException{ - List packets = new ArrayList<>(); - assert byteBuffer.hasArray(); - getDataChecksum().calculateChunkedSums(byteBuffer.array(), 0, - byteBuffer.remaining(), checksumBuf, 0); - int ckOff = 0; - while (byteBuffer.remaining() > 0) { - DFSPacket p = createPacket(packetSize, chunksPerPacket, - getCurrentStreamer().getBytesCurBlock(), - getCurrentStreamer().getAndIncCurrentSeqno(), false); - int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum; - int toWrite = byteBuffer.remaining() > maxBytesToPacket ? - maxBytesToPacket: byteBuffer.remaining(); - int chunks = (toWrite - 1) / bytesPerChecksum + 1; - int ckLen = chunks * getChecksumSize(); - p.writeChecksum(checksumBuf, ckOff, ckLen); - ckOff += ckLen; - p.writeData(byteBuffer, toWrite); - getCurrentStreamer().incBytesCurBlock(toWrite); - p.incNumChunks(chunks); - packets.add(p); - } - return packets; - } - @Override protected synchronized void writeChunk(byte[] bytes, int offset, int len, byte[] checksum, int ckoff, int cklen) throws IOException { @@ -413,11 +389,6 @@ protected synchronized void writeChunk(byte[] bytes, int offset, int len, if (!current.isFailed()) { try { super.writeChunk(bytes, offset, len, checksum, ckoff, cklen); - - // cell is full and current packet has not been enqueued, - if (cellFull && currentPacket != null) { - enqueueCurrentPacketFull(); - } } catch(Exception e) { handleStreamerFailure("offset=" + offset + ", length=" + len, e); } @@ -581,10 +552,14 @@ void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf final long oldBytes = current.getBytesCurBlock(); if (!current.isFailed()) { try { - for (DFSPacket p : generatePackets(buffer, checksumBuf)) { - getCurrentStreamer().waitAndQueuePacket(p); + DataChecksum sum = getDataChecksum(); + sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0); + for (int i = 0; i < len; i += sum.getBytesPerChecksum()) { + int chunkLen = Math.min(sum.getBytesPerChecksum(), len - i); + int ckOffset = i / sum.getBytesPerChecksum() * getChecksumSize(); + super.writeChunk(buffer.array(), i, chunkLen, checksumBuf, ckOffset, + getChecksumSize()); } - endBlock(); } catch(Exception e) { handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e); } @@ -628,16 +603,13 @@ protected synchronized void closeImpl() throws IOException { // flush from all upper layers try { flushBuffer(); - if (currentPacket != null) { - enqueueCurrentPacket(); - } + // if the last stripe is incomplete, generate and write parity cells + writeParityCellsForLastStripe(); + enqueueAllCurrentPackets(); } catch(Exception e) { handleStreamerFailure("closeImpl", e); } - // if the last stripe is incomplete, generate and write parity cells - writeParityCellsForLastStripe(); - for (int i = 0; i < numAllBlocks; i++) { final StripedDataStreamer s = setCurrentStreamer(i); if (!s.isFailed()) { @@ -667,4 +639,15 @@ protected synchronized void closeImpl() throws IOException { setClosed(); } } + + private void enqueueAllCurrentPackets() throws IOException { + int idx = streamers.indexOf(getCurrentStreamer()); + for(int i = 0; i < streamers.size(); i++) { + setCurrentStreamer(i); + if (currentPacket != null) { + enqueueCurrentPacket(); + } + } + setCurrentStreamer(idx); + } } From f32d9a175837c5b6c3d008089e46475d27a0935c Mon Sep 17 00:00:00 2001 From: yliu Date: Wed, 15 Jul 2015 22:35:19 +0800 Subject: [PATCH 178/212] HDFS-8619. Erasure Coding: revisit replica counting for striped blocks. (Jing Zhao via yliu) --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockInfo.java | 3 + .../blockmanagement/BlockInfoContiguous.java | 5 ++ .../blockmanagement/BlockInfoStriped.java | 11 ++++ .../server/blockmanagement/BlockManager.java | 42 ++++++------ .../server/blockmanagement/BlocksMap.java | 4 +- .../hdfs/TestReadStripedFileWithDecoding.java | 64 +++++++++++++++++++ .../server/namenode/TestAddStripedBlocks.java | 23 +++++-- 8 files changed, 127 insertions(+), 28 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 93c3162f352ac..8403d1adc90a5 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -353,3 +353,6 @@ HDFS-8734. Erasure Coding: fix one cell need two packets. (Walter Su via jing9) + + HDFS-8619. Erasure Coding: revisit replica counting for striped blocks. + (Jing Zhao via yliu) 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 61068b96dc768..82aa3489cd67d 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 @@ -178,6 +178,9 @@ public int getCapacity() { public abstract boolean isStriped(); + /** @return true if there is no datanode storage associated with the block */ + abstract boolean hasNoStorage(); + /** * Find specified DatanodeDescriptor. * @return index or -1 if not found. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index 5199101267d47..dfca8ea45af86 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -150,4 +150,9 @@ public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction( public final boolean isStriped() { return false; } + + @Override + final boolean hasNoStorage() { + return getStorageInfo(0) == null; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 07e29f8a7132a..66745101c084a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -272,4 +272,15 @@ public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( } return ucBlock; } + + @Override + final boolean hasNoStorage() { + final int len = getCapacity(); + for(int idx = 0; idx < len; idx++) { + if (getStorageInfo(idx) != null) { + return false; + } + } + return true; + } } 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 1aaf22569c4c1..b641fa2ac580c 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 @@ -1235,10 +1235,11 @@ public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk, } /** - * - * @param b + * Mark a replica (of a contiguous block) or an internal block (of a striped + * block group) as corrupt. + * @param b Indicating the reported bad block and the corresponding BlockInfo + * stored in blocksMap. * @param storageInfo storage that contains the block, if known. null otherwise. - * @throws IOException */ private void markBlockAsCorrupt(BlockToMarkCorrupt b, DatanodeStorageInfo storageInfo, @@ -1258,8 +1259,13 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, storageInfo.addBlock(b.stored, b.corrupted); } - // Add this replica to corruptReplicas Map - corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason, + // Add this replica to corruptReplicas Map. For striped blocks, we always + // use the id of whole striped block group when adding to corruptReplicas + Block corrupted = new Block(b.corrupted); + if (b.stored.isStriped()) { + corrupted.setBlockId(b.stored.getBlockId()); + } + corruptReplicas.addToCorruptReplicasMap(corrupted, node, b.reason, b.reasonCode); NumberReplicas numberOfReplicas = countNodes(b.stored); @@ -1283,7 +1289,7 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, if (hasEnoughLiveReplicas || hasMoreCorruptReplicas || corruptedDuringWrite) { // the block is over-replicated so invalidate the replicas immediately - invalidateBlock(b, node); + invalidateBlock(b, node, numberOfReplicas); } else if (namesystem.isPopulatingReplQueues()) { // add the block to neededReplication updateNeededReplications(b.stored, -1, 0); @@ -1295,8 +1301,8 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, * @return true if the block was successfully invalidated and no longer * present in the BlocksMap */ - private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn - ) throws IOException { + private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn, + NumberReplicas nr) throws IOException { blockLog.info("BLOCK* invalidateBlock: {} on {}", b, dn); DatanodeDescriptor node = getDatanodeManager().getDatanode(dn); if (node == null) { @@ -1305,7 +1311,6 @@ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn } // Check how many copies we have of the block - NumberReplicas nr = countNodes(b.stored); if (nr.replicasOnStaleNodes() > 0) { blockLog.info("BLOCK* invalidateBlocks: postponing " + "invalidation of {} on {} because {} replica(s) are located on " + @@ -1313,17 +1318,14 @@ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn nr.replicasOnStaleNodes()); postponeBlock(b.corrupted); return false; - } else if (nr.liveReplicas() >= 1) { - // If we have at least one copy on a live node, then we can delete it. + } else { + // we already checked the number of replicas in the caller of this + // function and know there are enough live replicas, so we can delete it. addToInvalidates(b.corrupted, dn); removeStoredBlock(b.stored, node); blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.", b, dn); return true; - } else { - blockLog.info("BLOCK* invalidateBlocks: {} on {} is the only copy and" + - " was not deleted", b, dn); - return false; } } @@ -2782,7 +2784,7 @@ private Block addStoredBlock(final BlockInfo block, " but corrupt replicas map has " + corruptReplicasCount); } if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) { - invalidateCorruptReplicas(storedBlock, reportedBlock); + invalidateCorruptReplicas(storedBlock, reportedBlock, num); } return storedBlock; } @@ -2814,18 +2816,20 @@ private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) { * * @param blk Block whose corrupt replicas need to be invalidated */ - private void invalidateCorruptReplicas(BlockInfo blk, Block reported) { + private void invalidateCorruptReplicas(BlockInfo blk, Block reported, + NumberReplicas numberReplicas) { Collection nodes = corruptReplicas.getNodes(blk); boolean removedFromBlocksMap = true; if (nodes == null) return; // make a copy of the array of nodes in order to avoid // ConcurrentModificationException, when the block is removed from the node - DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]); + DatanodeDescriptor[] nodesCopy = + nodes.toArray(new DatanodeDescriptor[nodes.size()]); for (DatanodeDescriptor node : nodesCopy) { try { if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null, - Reason.ANY), node)) { + Reason.ANY), node, numberReplicas)) { removedFromBlocksMap = false; } } catch (IOException e) { 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 9caf47c73fe79..91739200bf9b0 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 @@ -201,8 +201,8 @@ boolean removeNode(Block b, DatanodeDescriptor node) { // remove block from the data-node list and the node from the block info boolean removed = node.removeBlock(info); - if (info.getDatanode(0) == null // no datanodes left - && info.isDeleted()) { // does not belong to a file + if (info.hasNoStorage() // no datanodes left + && info.isDeleted()) { // does not belong to a file blocks.remove(b); // remove block from the map } return removed; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 3125e2e8aa883..34d6034210f7c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -24,7 +24,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; import org.junit.Assert; @@ -234,4 +238,64 @@ private void testReadWithDNFailure(String file, int fileSize, fileSize, readLen); Assert.assertArrayEquals(bytes, result.array()); } + + /** + * After reading a corrupted block, make sure the client can correctly report + * the corruption to the NameNode. + */ + @Test + public void testReportBadBlock() throws IOException { + // create file + final Path file = new Path("/corrupted"); + final int length = 10; // length of "corruption" + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, file, bytes); + + // corrupt the first data block + int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); + Assert.assertNotEquals(-1, dnIndex); + LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() + .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, + cellSize, dataBlocks, parityBlocks); + // find the first block file + File storageDir = cluster.getInstanceStorageDir(dnIndex, 0); + File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock()); + Assert.assertTrue("Block file does not exist", blkFile.exists()); + // corrupt the block file + LOG.info("Deliberately corrupting file " + blkFile.getName()); + try (FileOutputStream out = new FileOutputStream(blkFile)) { + out.write("corruption".getBytes()); + } + + // disable the heartbeat from DN so that the corrupted block record is kept + // in NameNode + for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); + } + + // do stateful read + ByteBuffer result = ByteBuffer.allocate(length); + ByteBuffer buf = ByteBuffer.allocate(1024); + int readLen = 0; + int ret; + try (FSDataInputStream in = fs.open(file)) { + while ((ret = in.read(buf)) >= 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } + Assert.assertEquals("The length of file should be the same to write size", + length, readLen); + Assert.assertArrayEquals(bytes, result.array()); + + // check whether the corruption has been reported to the NameNode + final FSNamesystem ns = cluster.getNamesystem(); + final BlockManager bm = ns.getBlockManager(); + BlockInfo blockInfo = (ns.getFSDirectory().getINode4Write(file.toString()) + .asFile().getBlocks())[0]; + Assert.assertEquals(1, bm.getCorruptReplicas(blockInfo).size()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 7876d1a1d2f90..8128772f36fdc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -325,6 +326,7 @@ public void testCheckStripedReplicaCorrupt() throws Exception { final int numStripes = 4; final Path filePath = new Path("/corrupt"); final FSNamesystem ns = cluster.getNameNode().getNamesystem(); + final BlockManager bm = ns.getBlockManager(); DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, numStripes, false); @@ -375,7 +377,10 @@ public void testCheckStripedReplicaCorrupt() throws Exception { ns.processIncrementalBlockReport( cluster.getDataNodes().get(3).getDatanodeId(), reports[0]); BlockManagerTestUtil.updateState(ns.getBlockManager()); - Assert.assertEquals(2, ns.getCorruptReplicaBlocks()); + // the total number of corrupted block info is still 1 + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + // 2 internal blocks corrupted + Assert.assertEquals(2, bm.getCorruptReplicas(stored).size()); // Now change the size of stored block, and test verifying the last // block size @@ -385,9 +390,10 @@ public void testCheckStripedReplicaCorrupt() throws Exception { reports = DFSTestUtil.makeReportForReceivedBlock(reported, ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); ns.processIncrementalBlockReport( - cluster.getDataNodes().get(3).getDatanodeId(), reports[0]); + cluster.getDataNodes().get(4).getDatanodeId(), reports[0]); BlockManagerTestUtil.updateState(ns.getBlockManager()); - Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); // Now send a parity block report with correct size based on adjusted // size of stored block @@ -400,16 +406,18 @@ public void testCheckStripedReplicaCorrupt() throws Exception { ns.processIncrementalBlockReport( cluster.getDataNodes().get(0).getDatanodeId(), reports[0]); BlockManagerTestUtil.updateState(ns.getBlockManager()); - Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); reported.setBlockId(stored.getBlockId() + 1); reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10); reports = DFSTestUtil.makeReportForReceivedBlock(reported, ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); ns.processIncrementalBlockReport( - cluster.getDataNodes().get(1).getDatanodeId(), reports[0]); + cluster.getDataNodes().get(5).getDatanodeId(), reports[0]); BlockManagerTestUtil.updateState(ns.getBlockManager()); - Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE); @@ -418,7 +426,8 @@ public void testCheckStripedReplicaCorrupt() throws Exception { ns.processIncrementalBlockReport( cluster.getDataNodes().get(2).getDatanodeId(), reports[0]); BlockManagerTestUtil.updateState(ns.getBlockManager()); - Assert.assertEquals(3, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); } } From 7e091de1366f4b57b5433bc19d738199dc05313d Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 15 Jul 2015 09:49:32 -0700 Subject: [PATCH 179/212] HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous blocks in INodeFile. Contributed by Zhe Zhang and Yi Liu. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockInfo.java | 1 + .../blockmanagement/BlockInfoContiguous.java | 1 - .../server/blockmanagement/BlockManager.java | 6 +- .../hdfs/server/namenode/FSDirAttrOp.java | 8 +- .../hdfs/server/namenode/FSDirConcatOp.java | 2 +- .../server/namenode/FSDirWriteFileOp.java | 59 ++- .../hdfs/server/namenode/FSImageFormat.java | 5 +- .../server/namenode/FSImageFormatPBINode.java | 76 ++-- .../FileWithStripedBlocksFeature.java | 116 ------ .../hdfs/server/namenode/INodeFile.java | 367 +++++++----------- .../server/namenode/INodeFileAttributes.java | 12 +- .../snapshot/FSImageFormatPBSnapshot.java | 3 +- .../server/namenode/snapshot/FileDiff.java | 9 +- .../namenode/snapshot/FileDiffList.java | 18 +- .../snapshot/FileWithSnapshotFeature.java | 4 +- .../hadoop-hdfs/src/main/proto/fsimage.proto | 8 +- .../hdfs/server/namenode/CreateEditsLog.java | 5 +- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../hdfs/server/namenode/TestEditLog.java | 4 +- .../server/namenode/TestFSEditLogLoader.java | 22 +- .../hdfs/server/namenode/TestFSImage.java | 16 +- .../hdfs/server/namenode/TestINodeFile.java | 2 +- .../server/namenode/TestStripedINodeFile.java | 14 +- .../namenode/TestTruncateQuotaUpdate.java | 4 +- 25 files changed, 273 insertions(+), 494 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8403d1adc90a5..e2ccd9ba8640e 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -356,3 +356,6 @@ HDFS-8619. Erasure Coding: revisit replica counting for striped blocks. (Jing Zhao via yliu) + + HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous + blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz) 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 82aa3489cd67d..2b823f51cc1de 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 @@ -31,6 +31,7 @@ */ public abstract class BlockInfo extends Block implements LightWeightGSet.LinkedElement { + public static final BlockInfo[] EMPTY_ARRAY = {}; private BlockCollection bc; /** For implementing {@link LightWeightGSet.LinkedElement} interface */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index dfca8ea45af86..bb9bf5b27bd8a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -26,7 +26,6 @@ */ @InterfaceAudience.Private public class BlockInfoContiguous extends BlockInfo { - public static final BlockInfoContiguous[] EMPTY_ARRAY = {}; public BlockInfoContiguous(short size) { super(size); 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 b641fa2ac580c..57904dfad8019 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 @@ -3037,13 +3037,13 @@ private MisReplicationResult processMisReplicatedBlock(BlockInfo block) { /** Set replication for the blocks. */ public void setReplication(final short oldRepl, final short newRepl, - final String src, final BlockInfoContiguous... blocks) { + final String src, final BlockInfo... blocks) { if (newRepl == oldRepl) { return; } // update needReplication priority queues - for(BlockInfoContiguous b : blocks) { + for(BlockInfo b : blocks) { updateNeededReplications(b, 0, newRepl-oldRepl); } @@ -3051,7 +3051,7 @@ public void setReplication(final short oldRepl, final short newRepl, // old replication > the new one; need to remove copies LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl + " for " + src); - for(BlockInfoContiguous b : blocks) { + for(BlockInfo b : blocks) { processOverReplicatedBlock(b, newRepl, null, null); } } else { // replication factor is increased diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index d34a0fa694669..796849b21fdc4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -25,14 +25,12 @@ import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.util.EnumCounters; @@ -150,7 +148,7 @@ static boolean setReplication( } final short[] blockRepls = new short[2]; // 0: old, 1: new - final BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src, + final BlockInfo[] blocks = unprotectedSetReplication(fsd, src, replication, blockRepls); isFile = blocks != null; if (isFile) { @@ -377,7 +375,7 @@ static INodeDirectory unprotectedSetQuota( } } - static BlockInfoContiguous[] unprotectedSetReplication( + static BlockInfo[] unprotectedSetReplication( FSDirectory fsd, String src, short replication, short[] blockRepls) throws QuotaExceededException, UnresolvedLinkException, SnapshotAccessControlException, UnsupportedActionException { @@ -417,7 +415,7 @@ static BlockInfoContiguous[] unprotectedSetReplication( blockRepls[0] = oldBR; blockRepls[1] = newBR; } - return file.getContiguousBlocks(); + return file.getBlocks(); } static void unprotectedSetStoragePolicy( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java index de5b1f00af6a5..750327218c1a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java @@ -234,7 +234,7 @@ static void unprotectedConcat(FSDirectory fsd, INodesInPath targetIIP, int count = 0; for (INodeFile nodeToRemove : srcList) { if(nodeToRemove != null) { - nodeToRemove.setContiguousBlocks(null); + nodeToRemove.clearBlocks(); nodeToRemove.getParent().removeChild(nodeToRemove); fsd.getINodeMap().remove(nodeToRemove); count++; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index e92388bcd6ca5..1d2b23c1e6984 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; @@ -481,25 +480,24 @@ static INodeFile addFileForEditLog( long preferredBlockSize, boolean underConstruction, String clientName, String clientMachine, byte storagePolicyId) { final INodeFile newNode; + Preconditions.checkNotNull(existing); assert fsd.hasWriteLock(); - if (underConstruction) { - newNode = newINodeFile(id, permissions, modificationTime, - modificationTime, replication, preferredBlockSize, storagePolicyId); - newNode.toUnderConstruction(clientName, clientMachine); - } else { - newNode = newINodeFile(id, permissions, modificationTime, atime, - replication, preferredBlockSize, storagePolicyId); - } - - newNode.setLocalName(localName); try { + // check if the file is in an EC zone + final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone( + fsd.getFSNamesystem(), existing); + if (underConstruction) { + newNode = newINodeFile(id, permissions, modificationTime, + modificationTime, replication, preferredBlockSize, storagePolicyId, + isStriped); + newNode.toUnderConstruction(clientName, clientMachine); + } else { + newNode = newINodeFile(id, permissions, modificationTime, atime, + replication, preferredBlockSize, storagePolicyId, isStriped); + } + newNode.setLocalName(localName); INodesInPath iip = fsd.addINode(existing, newNode); if (iip != null) { - // check if the file is in an EC zone - if (FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(), - iip)) { - newNode.addStripedBlocksFeature(); - } if (aclEntries != null) { AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID); } @@ -580,21 +578,18 @@ private static INodesInPath addFile( String clientName, String clientMachine) throws IOException { + Preconditions.checkNotNull(existing); long modTime = now(); - INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions, - modTime, modTime, replication, preferredBlockSize); - newNode.setLocalName(localName.getBytes(Charsets.UTF_8)); - newNode.toUnderConstruction(clientName, clientMachine); - INodesInPath newiip; fsd.writeLock(); try { + final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone( + fsd.getFSNamesystem(), existing); + INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions, + modTime, modTime, replication, preferredBlockSize, isStriped); + newNode.setLocalName(localName.getBytes(Charsets.UTF_8)); + newNode.toUnderConstruction(clientName, clientMachine); newiip = fsd.addINode(existing, newNode); - if (newiip != null - && FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(), - newiip)) { - newNode.addStripedBlocksFeature(); - } } finally { fsd.writeUnlock(); } @@ -788,16 +783,18 @@ private static boolean completeFileInternal( private static INodeFile newINodeFile( long id, PermissionStatus permissions, long mtime, long atime, - short replication, long preferredBlockSize, byte storagePolicyId) { + short replication, long preferredBlockSize, byte storagePolicyId, + boolean isStriped) { return new INodeFile(id, null, permissions, mtime, atime, - BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize, - storagePolicyId); + BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize, + storagePolicyId, isStriped); } private static INodeFile newINodeFile(long id, PermissionStatus permissions, - long mtime, long atime, short replication, long preferredBlockSize) { + long mtime, long atime, short replication, long preferredBlockSize, + boolean isStriped) { return newINodeFile(id, permissions, mtime, atime, replication, - preferredBlockSize, (byte)0); + preferredBlockSize, (byte)0, isStriped); } /** 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 d9a74e6fd5303..876427c3faa34 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 @@ -55,9 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; @@ -76,7 +74,6 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.StringUtils; import com.google.common.annotations.VisibleForTesting; @@ -903,7 +900,7 @@ public INodeFileAttributes loadINodeFileAttributes(DataInput in) final long preferredBlockSize = in.readLong(); return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime, - accessTime, replication, preferredBlockSize, (byte) 0, null); + accessTime, replication, preferredBlockSize, (byte) 0, null, false); } public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in) 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 1f8453911f4b2..013445e59d9cc 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 @@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto; -import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.StripedBlocksFeature; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto; @@ -326,13 +325,22 @@ private INodeFile loadINodeFile(INodeSection.INode n) { INodeSection.INodeFile f = n.getFile(); List bp = f.getBlocksList(); short replication = (short) f.getReplication(); + boolean isStriped = f.getIsStriped(); LoaderContext state = parent.getLoaderContext(); + ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); - BlockInfoContiguous[] blocks = null; - if (!f.hasStripedBlocks()) { - blocks = new BlockInfoContiguous[bp.size()]; - for (int i = 0, e = bp.size(); i < e; ++i) { - blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication); + if (isStriped) { + Preconditions.checkState(f.hasStripingCellSize()); + } + BlockInfo[] blocks = new BlockInfo[bp.size()]; + for (int i = 0; i < bp.size(); ++i) { + BlockProto b = bp.get(i); + if (isStriped) { + blocks[i] = new BlockInfoStriped(PBHelper.convert(b), schema, + (int)f.getStripingCellSize()); + } else { + blocks[i] = new BlockInfoContiguous(PBHelper.convert(b), + replication); } } @@ -342,46 +350,31 @@ private INodeFile loadINodeFile(INodeSection.INode n) { final INodeFile file = new INodeFile(n.getId(), n.getName().toByteArray(), permissions, f.getModificationTime(), f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(), - (byte)f.getStoragePolicyID()); + (byte)f.getStoragePolicyID(), isStriped); if (f.hasAcl()) { int[] entries = AclEntryStatusFormat.toInt(loadAclEntries( f.getAcl(), state.getStringTable())); file.addAclFeature(new AclFeature(entries)); } - + if (f.hasXAttrs()) { file.addXAttrFeature(new XAttrFeature( loadXAttrs(f.getXAttrs(), state.getStringTable()))); } - FileWithStripedBlocksFeature stripeFeature = null; - if (f.hasStripedBlocks()) { - // TODO: HDFS-7859 - ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); - stripeFeature = file.addStripedBlocksFeature(); - if (bp.size() > 0) { - // if a striped file has block, the cellSize must exist in proto - final int cellSize = f.getStripedBlocks().getCellSize(); - for (BlockProto b : bp) { - stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b), - schema, cellSize)); - } - } - } - // under-construction information if (f.hasFileUC()) { INodeSection.FileUnderConstructionFeature uc = f.getFileUC(); file.toUnderConstruction(uc.getClientName(), uc.getClientMachine()); - BlockInfo lastBlk = file.getLastBlock(); - if (lastBlk != null) { + if (blocks.length > 0) { + BlockInfo lastBlk = file.getLastBlock(); // replace the last block of file final BlockInfo ucBlk; - if (stripeFeature != null) { + if (isStriped) { BlockInfoStriped striped = (BlockInfoStriped) lastBlk; ucBlk = new BlockInfoStripedUnderConstruction(striped, - striped.getSchema(), striped.getCellSize()); + schema, (int)f.getStripingCellSize()); } else { ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication); @@ -500,7 +493,8 @@ public static INodeSection.INodeFile.Builder buildINodeFile( .setPermission(buildPermissionStatus(file, state.getStringMap())) .setPreferredBlockSize(file.getPreferredBlockSize()) .setReplication(file.getFileReplication()) - .setStoragePolicyID(file.getLocalStoragePolicyID()); + .setStoragePolicyID(file.getLocalStoragePolicyID()) + .setIsStriped(file.isStriped()); AclFeature f = file.getAclFeature(); if (f != null) { @@ -654,28 +648,22 @@ private void save(OutputStream out, INodeDirectory n) throws IOException { private void save(OutputStream out, INodeFile n) throws IOException { INodeSection.INodeFile.Builder b = buildINodeFile(n, parent.getSaverContext()); + BlockInfo[] blocks = n.getBlocks(); - BlockInfoContiguous[] cBlks = n.getContiguousBlocks(); - if (cBlks != null) { - for (Block block : cBlks) { + if (blocks != null) { + for (Block block : n.getBlocks()) { b.addBlocks(PBHelper.convert(block)); } } - FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature(); - if (sb != null) { - StripedBlocksFeature.Builder builder = - StripedBlocksFeature.newBuilder(); - BlockInfoStriped[] sblocks = sb.getBlocks(); - if (sblocks != null && sblocks.length > 0) { - final int cellSize = sblocks[0].getCellSize(); - for (BlockInfoStriped sblk : sblocks) { - assert cellSize == sblk.getCellSize(); - b.addBlocks(PBHelper.convert(sblk)); - } - builder.setCellSize(cellSize); + if (n.isStriped()) { + if (blocks != null && blocks.length > 0) { + BlockInfo firstBlock = blocks[0]; + Preconditions.checkState(firstBlock.isStriped()); + b.setStripingCellSize(((BlockInfoStriped)firstBlock).getCellSize()); + } else { + b.setStripingCellSize(HdfsConstants.BLOCK_STRIPED_CELL_SIZE); } - b.setStripedBlocks(builder.build()); } FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java deleted file mode 100644 index 94ab527e40c18..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java +++ /dev/null @@ -1,116 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hdfs.server.namenode; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; - -/** - * Feature for file with striped blocks - */ -class FileWithStripedBlocksFeature implements INode.Feature { - private BlockInfoStriped[] blocks; - - FileWithStripedBlocksFeature() { - blocks = new BlockInfoStriped[0]; - } - - FileWithStripedBlocksFeature(BlockInfoStriped[] blocks) { - Preconditions.checkArgument(blocks != null); - this.blocks = blocks; - } - - BlockInfoStriped[] getBlocks() { - return this.blocks; - } - - void setBlock(int index, BlockInfoStriped blk) { - blocks[index] = blk; - } - - BlockInfoStriped getLastBlock() { - return blocks == null || blocks.length == 0 ? - null : blocks[blocks.length - 1]; - } - - int numBlocks() { - return blocks == null ? 0 : blocks.length; - } - - void updateBlockCollection(INodeFile file) { - if (blocks != null) { - for (BlockInfoStriped blk : blocks) { - blk.setBlockCollection(file); - } - } - } - - private void setBlocks(BlockInfoStriped[] blocks) { - this.blocks = blocks; - } - - void addBlock(BlockInfoStriped newBlock) { - if (this.blocks == null) { - this.setBlocks(new BlockInfoStriped[]{newBlock}); - } else { - int size = this.blocks.length; - BlockInfoStriped[] newlist = new BlockInfoStriped[size + 1]; - System.arraycopy(this.blocks, 0, newlist, 0, size); - newlist[size] = newBlock; - this.setBlocks(newlist); - } - } - - BlockInfoStripedUnderConstruction removeLastBlock( - Block oldblock) { - if (blocks == null || blocks.length == 0) { - return null; - } - int newSize = blocks.length - 1; - if (!blocks[newSize].equals(oldblock)) { - return null; - } - - BlockInfoStripedUnderConstruction uc = - (BlockInfoStripedUnderConstruction) blocks[newSize]; - //copy to a new list - BlockInfoStriped[] newlist = new BlockInfoStriped[newSize]; - System.arraycopy(blocks, 0, newlist, 0, newSize); - setBlocks(newlist); - return uc; - } - - void truncateStripedBlocks(int n) { - final BlockInfoStriped[] newBlocks; - if (n == 0) { - newBlocks = new BlockInfoStriped[0]; - } else { - newBlocks = new BlockInfoStriped[n]; - System.arraycopy(getBlocks(), 0, newBlocks, 0, n); - } - // set new blocks - setBlocks(newBlocks); - } - - void clear() { - this.blocks = null; - } -} 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 b2b0fea8258a2..25415efb02388 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 @@ -40,9 +40,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -84,12 +82,14 @@ public static INodeFile valueOf(INode inode, String path, boolean acceptNull) /** * Bit format: - * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize] + * [4-bit storagePolicyID][1-bit isStriped] + * [11-bit replication][48-bit preferredBlockSize] */ static enum HeaderFormat { PREFERRED_BLOCK_SIZE(null, 48, 1), - REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0), - STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH, + REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0), + IS_STRIPED(REPLICATION.BITS, 1, 0), + STORAGE_POLICY_ID(IS_STRIPED.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH, 0); private final LongBitFormat BITS; @@ -110,14 +110,27 @@ static byte getStoragePolicyID(long header) { return (byte)STORAGE_POLICY_ID.BITS.retrieve(header); } + static boolean isStriped(long header) { + long isStriped = IS_STRIPED.BITS.retrieve(header); + Preconditions.checkState(isStriped == 0 || isStriped == 1); + return isStriped == 1; + } + static long toLong(long preferredBlockSize, short replication, - byte storagePolicyID) { + boolean isStriped, byte storagePolicyID) { long h = 0; if (preferredBlockSize == 0) { preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin(); } h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h); - h = REPLICATION.BITS.combine(replication, h); + // Replication factor for striped files is zero + if (isStriped) { + h = REPLICATION.BITS.combine(0L, h); + h = IS_STRIPED.BITS.combine(1L, h); + } else { + h = REPLICATION.BITS.combine(replication, h); + h = IS_STRIPED.BITS.combine(0L, h); + } h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h); return h; } @@ -126,21 +139,27 @@ static long toLong(long preferredBlockSize, short replication, 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); + preferredBlockSize, (byte) 0, false); } INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, - long atime, BlockInfoContiguous[] blklist, short replication, - long preferredBlockSize, byte storagePolicyID) { + long atime, BlockInfo[] blklist, short replication, + long preferredBlockSize, byte storagePolicyID, boolean isStriped) { super(id, name, permissions, mtime, atime); - header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID); - this.blocks = blklist; + header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped, + storagePolicyID); + if (blklist != null && blklist.length > 0) { + for (BlockInfo b : blklist) { + Preconditions.checkArgument(b.isStriped() == isStriped); + } + } + setBlocks(blklist); } public INodeFile(INodeFile that) { @@ -177,27 +196,6 @@ && getAclFeature() == other.getAclFeature() && getXAttrFeature() == other.getXAttrFeature(); } - /* Start of StripedBlock Feature */ - - public final FileWithStripedBlocksFeature getStripedBlocksFeature() { - return getFeature(FileWithStripedBlocksFeature.class); - } - - public FileWithStripedBlocksFeature addStripedBlocksFeature() { - assert blocks == null || blocks.length == 0: - "The file contains contiguous blocks"; - assert !isStriped(); - this.setFileReplication((short) 0); - FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature(); - addFeature(sb); - return sb; - } - - /** Used to make sure there is no contiguous block related info */ - private boolean hasNoContiguousBlock() { - return (blocks == null || blocks.length == 0) && getFileReplication() == 0; - } - /* Start of Under-Construction Feature */ /** @@ -232,7 +230,7 @@ public INodeFile toCompleteFile(long mtime) { "file is no longer under construction"); FileUnderConstructionFeature uc = getFileUnderConstructionFeature(); if (uc != null) { - assertAllBlocksComplete(getBlocks()); + assertAllBlocksComplete(); removeFeature(uc); this.setModificationTime(mtime); } @@ -240,32 +238,21 @@ public INodeFile toCompleteFile(long mtime) { } /** Assert all blocks are complete. */ - private void assertAllBlocksComplete(BlockInfo[] blks) { - if (blks == null) { + private void assertAllBlocksComplete() { + if (blocks == null) { return; } - for (int i = 0; i < blks.length; i++) { - Preconditions.checkState(blks[i].isComplete(), "Failed to finalize" + for (int i = 0; i < blocks.length; i++) { + Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize" + " %s %s since blocks[%s] is non-complete, where blocks=%s.", - getClass().getSimpleName(), this, i, Arrays.asList(blks)); + getClass().getSimpleName(), this, i, Arrays.asList(blocks)); } } - /** - * Instead of adding a new block, this function is usually used while loading - * fsimage or converting the last block to UC/Complete. - */ @Override // BlockCollection public void setBlock(int index, BlockInfo blk) { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - assert !blk.isStriped(); - this.blocks[index] = (BlockInfoContiguous) blk; - } else { - assert blk.isStriped(); - assert hasNoContiguousBlock(); - sb.setBlock(index, (BlockInfoStriped) blk); - } + Preconditions.checkArgument(blk.isStriped() == this.isStriped()); + this.blocks[index] = blk; } @Override // BlockCollection, the file should be under construction @@ -278,15 +265,13 @@ public void convertLastBlockToUC(BlockInfo lastBlock, } final BlockInfo ucBlock; - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - assert !lastBlock.isStriped(); - ucBlock = ((BlockInfoContiguous) lastBlock) + if (isStriped()) { + Preconditions.checkState(lastBlock.isStriped()); + ucBlock = ((BlockInfoStriped) lastBlock) .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); } else { - assert hasNoContiguousBlock(); - assert lastBlock.isStriped(); - ucBlock = ((BlockInfoStriped) lastBlock) + Preconditions.checkState(!lastBlock.isStriped()); + ucBlock = ((BlockInfoContiguous) lastBlock) .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); } setBlock(numBlocks() - 1, ucBlock); @@ -299,27 +284,21 @@ public void convertLastBlockToUC(BlockInfo lastBlock, BlockInfoUnderConstruction removeLastBlock(Block oldblock) { Preconditions.checkState(isUnderConstruction(), "file is no longer under construction"); - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - if (blocks == null || blocks.length == 0) { - return null; - } - int size_1 = blocks.length - 1; - if (!blocks[size_1].equals(oldblock)) { - return null; - } - - BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction)blocks[size_1]; - //copy to a new list - BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1]; - System.arraycopy(blocks, 0, newlist, 0, size_1); - setContiguousBlocks(newlist); - return uc; - } else { - assert hasNoContiguousBlock(); - return sb.removeLastBlock(oldblock); + if (blocks == null || blocks.length == 0) { + return null; + } + int size_1 = blocks.length - 1; + if (!blocks[size_1].equals(oldblock)) { + return null; } + + BlockInfoUnderConstruction uc = + (BlockInfoUnderConstruction)blocks[size_1]; + //copy to a new list + BlockInfo[] newlist = new BlockInfo[size_1]; + System.arraycopy(blocks, 0, newlist, 0, size_1); + setBlocks(newlist); + return uc; } /* End of Under-Construction Feature */ @@ -401,7 +380,7 @@ public final short getFileReplication(int snapshot) { /** The same as getFileReplication(null). */ @Override // INodeFileAttributes - // TODO striped + // TODO properly handle striped files public final short getFileReplication() { return getFileReplication(CURRENT_STATE_ID); } @@ -466,6 +445,16 @@ public final void setStoragePolicyID(byte storagePolicyId, setStoragePolicyID(storagePolicyId); } + + /** + * @return true if the file is in the striping layout. + */ + @VisibleForTesting + @Override + public boolean isStriped() { + return HeaderFormat.isStriped(header); + } + @Override // INodeFileAttributes public long getHeaderLong() { return header; @@ -474,17 +463,6 @@ public long getHeaderLong() { /** @return the blocks of the file. */ @Override // BlockCollection public BlockInfo[] getBlocks() { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb != null) { - assert hasNoContiguousBlock(); - return sb.getBlocks(); - } else { - return this.blocks; - } - } - - /** Used by snapshot diff */ - public BlockInfoContiguous[] getContiguousBlocks() { return this.blocks; } @@ -507,16 +485,11 @@ public BlockInfo[] getBlocks(int snapshot) { } /** Used during concat to update the BlockCollection for each block */ - private void updateBlockCollection() { - if (blocks != null && blocks.length > 0) { - for(BlockInfoContiguous b : blocks) { + void updateBlockCollection() { + if (blocks != null) { + for(BlockInfo b : blocks) { b.setBlockCollection(this); } - } else { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb != null) { - sb.updateBlockCollection(this); - } } } @@ -527,11 +500,12 @@ void concatBlocks(INodeFile[] inodes) { int size = this.blocks.length; int totalAddedBlocks = 0; for(INodeFile f : inodes) { + Preconditions.checkState(f.isStriped() == this.isStriped()); 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) { @@ -539,43 +513,35 @@ void concatBlocks(INodeFile[] inodes) { size += in.blocks.length; } - setContiguousBlocks(newlist); + setBlocks(newlist); updateBlockCollection(); } /** - * add a contiguous block to the block list + * add a block to the block list */ - private void addContiguousBlock(BlockInfoContiguous newblock) { + void addBlock(BlockInfo newblock) { + Preconditions.checkArgument(newblock.isStriped() == this.isStriped()); if (this.blocks == null) { - this.setContiguousBlocks(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.setContiguousBlocks(newlist); - } - } - - /** add a striped or contiguous block */ - void addBlock(BlockInfo newblock) { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - assert !newblock.isStriped(); - addContiguousBlock((BlockInfoContiguous) newblock); - } else { - assert newblock.isStriped(); - assert hasNoContiguousBlock(); - sb.addBlock((BlockInfoStriped) newblock); + this.setBlocks(newlist); } } /** Set the blocks. */ - public void setContiguousBlocks(BlockInfoContiguous[] blocks) { + private void setBlocks(BlockInfo[] blocks) { this.blocks = blocks; } + public void clearBlocks() { + setBlocks(null); + } + @Override public void cleanSubtree(ReclaimContext reclaimContext, final int snapshot, int priorSnapshotId) { @@ -623,19 +589,13 @@ public void destroyAndCollectBlocks(ReclaimContext reclaimContext) { } public void clearFile(ReclaimContext reclaimContext) { - BlockInfo[] blks = getBlocks(); - if (blks != null && reclaimContext.collectedBlocks != null) { - for (BlockInfo blk : blks) { + if (blocks != null && reclaimContext.collectedBlocks != null) { + for (BlockInfo blk : blocks) { reclaimContext.collectedBlocks.addDeleteBlock(blk); blk.setBlockCollection(null); } } - setContiguousBlocks(null); - - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb != null) { - sb.clear(); - } + clearBlocks(); if (getAclFeature() != null) { AclStorage.removeAclFeature(getAclFeature()); } @@ -677,7 +637,7 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, if (isStriped()) { return computeQuotaUsageWithStriped(bsp, counts); } - + if (last < lastSnapshotId) { ssDeltaNoReplication = computeFileSize(true, false); replication = getFileReplication(); @@ -790,33 +750,27 @@ public final long computeFileSizeNotIncludingLastUcBlock() { * @return file size */ public final long computeFileSize(boolean includesLastUcBlock, - boolean usePreferredBlockSize4LastUcBlock) { - BlockInfo[] blockInfos = getBlocks(); - // In case of contiguous blocks - if (blockInfos == null || blockInfos.length == 0) { + boolean usePreferredBlockSize4LastUcBlock) { + if (blocks == null || blocks.length == 0) { return 0; } - final int last = blockInfos.length - 1; + final int last = blocks.length - 1; //check if the last block is BlockInfoUnderConstruction - long size = blockInfos[last].getNumBytes(); - if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) { - if (!includesLastUcBlock) { - size = 0; - } else if (usePreferredBlockSize4LastUcBlock) { - size = getPreferredBlockSize(); - } - } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) { - if (!includesLastUcBlock) { - size = 0; - } else if (usePreferredBlockSize4LastUcBlock) { - BlockInfoStripedUnderConstruction blockInfoStripedUC - = (BlockInfoStripedUnderConstruction) blockInfos[last]; - size = getPreferredBlockSize() * blockInfoStripedUC.getDataBlockNum(); - } + BlockInfo lastBlk = blocks[last]; + long size = lastBlk.getNumBytes(); + if (lastBlk instanceof BlockInfoUnderConstruction) { + if (!includesLastUcBlock) { + size = 0; + } else if (usePreferredBlockSize4LastUcBlock) { + size = isStriped()? + getPreferredBlockSize() * + ((BlockInfoStriped)lastBlk).getDataBlockNum() : + getPreferredBlockSize(); + } } //sum other blocks for (int i = 0; i < last; i++) { - size += blockInfos[i].getNumBytes(); + size += blocks[i].getNumBytes(); } return size; } @@ -828,41 +782,30 @@ public final long computeFileSize(boolean includesLastUcBlock, */ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { if (isStriped()) { - return storagespaceConsumedWithStriped(); + return storagespaceConsumedStriped(); } else { - return storagespaceConsumedWithReplication(bsp); + return storagespaceConsumedContiguous(bsp); } } // TODO: support EC with heterogeneous storage - public final QuotaCounts storagespaceConsumedWithStriped() { + public final QuotaCounts storagespaceConsumedStriped() { QuotaCounts counts = new QuotaCounts.Builder().build(); - BlockInfo[] blockInfos = getBlocks(); - if (blockInfos == null || blockInfos.length == 0) { + if (blocks == null || blocks.length == 0) { return counts; } - long size; - final int last = blockInfos.length - 1; - if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) { - BlockInfoStripedUnderConstruction blockInfoStripedUC - =(BlockInfoStripedUnderConstruction)blockInfos[last]; - size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum(); - } else { - // In case of last block is complete - BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last]; - size = blockInfoStriped.spaceConsumed(); - } - for (int i = 0; i < last; i++) { - BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i]; - size += blockInfoStriped.spaceConsumed(); + for (BlockInfo b : blocks) { + Preconditions.checkState(b.isStriped()); + long blockSize = b.isComplete() ? + ((BlockInfoStriped)b).spaceConsumed() : getPreferredBlockSize() * + ((BlockInfoStriped)b).getTotalBlockNum(); + counts.addStorageSpace(blockSize); } - - counts.addStorageSpace(size); return counts; } - public final QuotaCounts storagespaceConsumedWithReplication( + public final QuotaCounts storagespaceConsumedContiguous( BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build(); final Iterable blocks; @@ -874,7 +817,7 @@ public final QuotaCounts storagespaceConsumedWithReplication( 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)); } @@ -903,32 +846,20 @@ public final QuotaCounts storagespaceConsumedWithReplication( * Return the penultimate allocated block for this file. */ BlockInfo getPenultimateBlock() { - BlockInfo[] blks = getBlocks(); - return (blks == null || blks.length <= 1) ? - null : blks[blks.length - 2]; + if (blocks == null || blocks.length <= 1) { + return null; + } + return blocks[blocks.length - 2]; } @Override public BlockInfo getLastBlock() { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - return blocks == null || blocks.length == 0 ? - null : blocks[blocks.length - 1]; - } else { - assert hasNoContiguousBlock(); - return sb.getLastBlock(); - } + return blocks == null || blocks.length == 0? null: blocks[blocks.length-1]; } @Override public int numBlocks() { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - return blocks == null ? 0 : blocks.length; - } else { - assert hasNoContiguousBlock(); - return sb.numBlocks(); - } + return blocks == null ? 0 : blocks.length; } @VisibleForTesting @@ -939,8 +870,7 @@ public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix, out.print(", fileSize=" + computeFileSize(snapshotId)); // only compare the first block out.print(", blocks="); - BlockInfo[] blks = getBlocks(); - out.print(blks == null || blks.length == 0? null: blks[0]); + out.print(blocks == null || blocks.length == 0? null: blocks[0]); out.println(); } @@ -1033,24 +963,15 @@ void computeQuotaDeltaForTruncate( } void truncateBlocksTo(int n) { - FileWithStripedBlocksFeature sb = getStripedBlocksFeature(); - if (sb == null) { - truncateContiguousBlocks(n); - } else { - sb.truncateStripedBlocks(n); - } - } - - private void truncateContiguousBlocks(int n) { - final BlockInfoContiguous[] newBlocks; + final BlockInfo[] newBlocks; if (n == 0) { - newBlocks = BlockInfoContiguous.EMPTY_ARRAY; + newBlocks = BlockInfo.EMPTY_ARRAY; } else { - newBlocks = new BlockInfoContiguous[n]; - System.arraycopy(blocks, 0, newBlocks, 0, n); + newBlocks = new BlockInfo[n]; + System.arraycopy(getBlocks(), 0, newBlocks, 0, n); } // set new blocks - setContiguousBlocks(newBlocks); + setBlocks(newBlocks); } /** @@ -1059,10 +980,11 @@ private void truncateContiguousBlocks(int n) { * snapshots. Since we do not support truncation with striped blocks, * we only need to handle contiguous blocks here. */ - public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks, + public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks, BlocksMapUpdateInfo collectedBlocks) { - BlockInfoContiguous[] oldBlocks = this.blocks; - if (snapshotBlocks == null || oldBlocks == null) + Preconditions.checkState(!isStriped()); + BlockInfo[] oldBlocks = getBlocks(); + if(snapshotBlocks == null || oldBlocks == null) return; // Skip blocks in common between the file and the snapshot int n = 0; @@ -1070,7 +992,7 @@ public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks, oldBlocks[n] == snapshotBlocks[n]) { n++; } - truncateContiguousBlocks(n); + truncateBlocksTo(n); // Collect the remaining blocks of the file while(n < oldBlocks.length) { collectedBlocks.addDeleteBlock(oldBlocks[n++]); @@ -1085,7 +1007,7 @@ void excludeSnapshotBlocks(int snapshotId, FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); if(sf == null) return; - BlockInfoContiguous[] snapshotBlocks = + BlockInfo[] snapshotBlocks = getDiffs().findEarlierSnapshotBlocks(snapshotId); if(snapshotBlocks == null) return; @@ -1099,23 +1021,14 @@ void excludeSnapshotBlocks(int snapshotId, /** * @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); } - - /** - * @return true if the file is in the striping layout. - */ - @VisibleForTesting - @Override - public boolean isStriped() { - return getStripedBlocksFeature() != null; - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java index 204c8ac6a8b67..13bd9e962e3fe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java @@ -29,6 +29,9 @@ public interface INodeFileAttributes extends INodeAttributes { /** @return the file replication. */ public short getFileReplication(); + /** @return whether the file is striped (instead of contiguous) */ + public boolean isStriped(); + /** @return preferred block size in bytes */ public long getPreferredBlockSize(); @@ -47,10 +50,10 @@ public static class SnapshotCopy extends INodeAttributes.SnapshotCopy public SnapshotCopy(byte[] name, PermissionStatus permissions, AclFeature aclFeature, long modificationTime, long accessTime, short replication, long preferredBlockSize, - byte storagePolicyID, XAttrFeature xAttrsFeature) { + byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) { super(name, permissions, aclFeature, modificationTime, accessTime, xAttrsFeature); - header = HeaderFormat.toLong(preferredBlockSize, replication, + header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped, storagePolicyID); } @@ -69,6 +72,11 @@ public short getFileReplication() { return HeaderFormat.getReplication(header); } + @Override + public boolean isStriped() { + return HeaderFormat.isStriped(header); + } + @Override public long getPreferredBlockSize() { return HeaderFormat.getPreferredBlockSize(header); 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 717c8fb00becf..0bcb860b9d8f0 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 @@ -233,7 +233,8 @@ private void loadFileDiffList(InputStream in, INodeFile file, int size) .toByteArray(), permission, acl, fileInPb.getModificationTime(), fileInPb.getAccessTime(), (short) fileInPb.getReplication(), fileInPb.getPreferredBlockSize(), - (byte)fileInPb.getStoragePolicyID(), xAttrs); + (byte)fileInPb.getStoragePolicyID(), xAttrs, + fileInPb.getIsStriped()); } FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, 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 48c5f334e1430..896e4ba09e662 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,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization; import org.apache.hadoop.hdfs.server.namenode.INode; @@ -38,7 +39,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 +65,7 @@ public long getFileSize() { * 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 +74,7 @@ public void setBlocks(BlockInfoContiguous[] blocks) { this.blocks = Arrays.copyOf(blocks, numBlocks); } - public BlockInfoContiguous[] getBlocks() { + public BlockInfo[] getBlocks() { return blocks; } @@ -118,7 +119,7 @@ public void destroyAndCollectSnapshotBlocks( if (blocks == null || collectedBlocks == null) { return; } - for (BlockInfoContiguous blk : blocks) { + for (BlockInfo blk : blocks) { collectedBlocks.addDeleteBlock(blk); } blocks = null; 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 a5c782ddafd8b..442e127cdc52b 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 @@ -56,20 +56,20 @@ public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile, final FileDiff diff = super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy); if (withBlocks) { // Store blocks if this is the first update - BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks(); + BlockInfo[] blks = iNodeFile.getBlocks(); assert blks != null; diff.setBlocks(blks); } } - 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) { @@ -79,14 +79,14 @@ public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) { 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) { @@ -103,7 +103,7 @@ public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) { */ 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"; @@ -117,11 +117,11 @@ void combineAndCollectSnapshotBlocks( if (earlierDiff != null) { earlierDiff.setBlocks(removedBlocks); } - BlockInfoContiguous[] earlierBlocks = + BlockInfo[] earlierBlocks = (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks()); // Find later snapshot (or file itself) with blocks - BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId()); - laterBlocks = (laterBlocks == null) ? file.getContiguousBlocks() : laterBlocks; + 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; for(; i < removedBlocks.length; i++) { 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 555a662b0131b..f017d4b7fb542 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 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 void collectBlocksAndClear( // 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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto index 5bb1c3242d2b1..3233f66e5240f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto @@ -92,11 +92,6 @@ message INodeSection { optional string clientMachine = 2; } - message StripedBlocksFeature { - // store striped blocks related information - optional uint32 cellSize = 1; - } - message AclFeatureProto { /** * An ACL entry is represented by a 32-bit integer in Big Endian @@ -145,7 +140,8 @@ message INodeSection { optional AclFeatureProto acl = 8; optional XAttrFeatureProto xAttrs = 9; optional uint32 storagePolicyID = 10; - optional StripedBlocksFeature stripedBlocks = 11; + optional bool isStriped = 11; + optional uint64 stripingCellSize = 12; } message QuotaByStorageTypeEntryProto { 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 0349251ad38c6..f7ff97dc26e44 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,6 +23,7 @@ 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.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.Storage; @@ -66,7 +67,7 @@ static void addFiles(FSEditLog editLog, int numFiles, short replication, 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), @@ -97,7 +98,7 @@ static void addFiles(FSEditLog editLog, int numFiles, short replication, 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 8128772f36fdc..b8aac71d25d7b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -333,7 +333,7 @@ public void testCheckStripedReplicaCorrupt() throws Exception { INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()). asFile(); Assert.assertTrue(fileNode.isStriped()); - BlockInfoStriped stored = fileNode.getStripedBlocksFeature().getBlocks()[0]; + BlockInfo stored = fileNode.getBlocks()[0]; BlockManagerTestUtil.updateState(ns.getBlockManager()); Assert.assertEquals(0, ns.getCorruptReplicaBlocks()); 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 1e42e34532d5a..f654107af6cee 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.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 void run() { 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 79cb4957c82b4..5180f1336b408 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -468,7 +469,7 @@ public void testAddNewStripedBlock() throws IOException{ new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); - file.getStripedBlocksFeature().addBlock(stripedBlk); + file.addBlock(stripedBlk); fns.getEditLog().logAddBlock(testFilePath, file); file.toCompleteFile(System.currentTimeMillis()); @@ -483,14 +484,14 @@ public void testAddNewStripedBlock() throws IOException{ assertTrue(inodeLoaded.isStriped()); - BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks(); + BlockInfo[] blks = inodeLoaded.getBlocks(); assertEquals(1, blks.length); assertEquals(blkId, blks[0].getBlockId()); assertEquals(blkNumBytes, blks[0].getNumBytes()); assertEquals(timestamp, blks[0].getGenerationStamp()); - assertEquals(blockNum, blks[0].getDataBlockNum()); - assertEquals(parityNum, blks[0].getParityBlockNum()); - assertEquals(cellSize, blks[0].getCellSize()); + assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum()); + assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize()); cluster.shutdown(); cluster = null; @@ -536,7 +537,7 @@ public void testUpdateStripedBlocks() throws IOException{ new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); - file.getStripedBlocksFeature().addBlock(stripedBlk); + file.addBlock(stripedBlk); fns.getEditLog().logAddBlock(testFilePath, file); file.toCompleteFile(System.currentTimeMillis()); fns.enterSafeMode(false); @@ -564,14 +565,15 @@ public void testUpdateStripedBlocks() throws IOException{ assertTrue(inodeLoaded.isStriped()); - BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks(); + BlockInfo[] blks = inodeLoaded.getBlocks(); assertEquals(1, blks.length); + assertTrue(blks[0].isStriped()); assertEquals(blkId, blks[0].getBlockId()); assertEquals(newBlkNumBytes, blks[0].getNumBytes()); assertEquals(newTimestamp, blks[0].getGenerationStamp()); - assertEquals(blockNum, blks[0].getDataBlockNum()); - assertEquals(parityNum, blks[0].getParityBlockNum()); - assertEquals(cellSize, blks[0].getCellSize()); + assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum()); + assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize()); cluster.shutdown(); cluster = null; 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 c55b9ba80fb52..a14492294fb4d 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 @@ -152,9 +152,8 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con short replication = 3; long preferredBlockSize = 128*1024*1024; INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime, - blks, replication, preferredBlockSize); + blks, replication, preferredBlockSize, (byte) 0, true); ByteArrayOutputStream bs = new ByteArrayOutputStream(); - file.addStripedBlocksFeature(); //construct StripedBlocks for the INode BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3]; @@ -164,7 +163,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con stripedBlks[i] = new BlockInfoStriped( new Block(stripedBlkId + i, preferredBlockSize, timestamp), testSchema, cellSize); - file.getStripedBlocksFeature().addBlock(stripedBlks[i]); + file.addBlock(stripedBlks[i]); } final String client = "testClient"; @@ -206,7 +205,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime()); // TODO for striped blocks, we currently save and load them as contiguous // blocks to/from legacy fsimage - assertEquals(3, fileByLoaded.getContiguousBlocks().length); + assertEquals(3, fileByLoaded.getBlocks().length); assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize()); if (isUC) { @@ -405,13 +404,12 @@ public void testSupportBlockGroup() throws IOException { // check the information of striped blocks FSNamesystem fsn = cluster.getNamesystem(); INodeFile inode = fsn.dir.getINode(file.toString()).asFile(); - FileWithStripedBlocksFeature sb = inode.getStripedBlocksFeature(); - assertNotNull(sb); - BlockInfoStriped[] blks = sb.getBlocks(); + assertTrue(inode.isStriped()); + BlockInfo[] blks = inode.getBlocks(); assertEquals(1, blks.length); assertTrue(blks[0].isStriped()); - assertEquals(HdfsConstants.NUM_DATA_BLOCKS, blks[0].getDataBlockNum()); - assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, blks[0].getParityBlockNum()); + assertEquals(HdfsConstants.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum()); } finally { cluster.shutdown(); } 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 b45d2f6dcf666..8dbde6b62458d 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 @@ -92,7 +92,7 @@ INodeFile createINodeFile(short replication, long preferredBlockSize) { private static INodeFile createINodeFile(byte storagePolicyID) { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, - null, (short)3, 1024L, storagePolicyID); + null, (short)3, 1024L, storagePolicyID, false); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index c0022bd519f44..477c609936f9a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -57,14 +57,13 @@ public class TestStripedINodeFile { private static INodeFile createStripedINodeFile() { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, - null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID); + null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID, true); } @Test public void testBlockStripedFeature() throws IOException, InterruptedException{ INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); assertTrue(inf.isStriped()); } @@ -80,7 +79,6 @@ public void testBlockStripedTotalBlockCount() { public void testBlockStripedLength() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped = new BlockInfoStriped(blk, testSchema, cellSize); @@ -92,7 +90,6 @@ public void testBlockStripedLength() public void testBlockStripedConsumedSpace() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped = new BlockInfoStriped(blk, testSchema, cellSize); @@ -110,7 +107,7 @@ public void testBlockStripedConsumedSpace() // a. * ( - 1) * = 0 // b. % = 1 // c. * = 1 * 3 - assertEquals(4, inf.storagespaceConsumedWithStriped().getStorageSpace()); + assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace()); assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace()); } @@ -118,7 +115,6 @@ public void testBlockStripedConsumedSpace() public void testMultipleBlockStripedConsumedSpace() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk1 = new Block(1); BlockInfoStriped blockInfoStriped1 = new BlockInfoStriped(blk1, testSchema, cellSize); @@ -130,7 +126,7 @@ public void testMultipleBlockStripedConsumedSpace() inf.addBlock(blockInfoStriped1); inf.addBlock(blockInfoStriped2); // This is the double size of one block in above case. - assertEquals(4 * 2, inf.storagespaceConsumedWithStriped().getStorageSpace()); + assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace()); assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace()); } @@ -138,7 +134,6 @@ public void testMultipleBlockStripedConsumedSpace() public void testBlockStripedFileSize() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped = new BlockInfoStriped(blk, testSchema, cellSize); @@ -154,7 +149,6 @@ public void testBlockStripedFileSize() public void testBlockStripedUCFileSize() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStripedUnderConstruction bInfoStripedUC = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); @@ -168,7 +162,6 @@ public void testBlockStripedUCFileSize() public void testBlockStripedComputeQuotaUsage() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped = new BlockInfoStriped(blk, testSchema, cellSize); @@ -190,7 +183,6 @@ public void testBlockStripedComputeQuotaUsage() public void testBlockStripedUCComputeQuotaUsage() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); - inf.addStripedBlocksFeature(); Block blk = new Block(1); BlockInfoStripedUnderConstruction bInfoStripedUC = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java index 57159dbec2c14..a13892e33e736 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java @@ -78,7 +78,7 @@ public void testTruncateWithoutSnapshot() { @Test public void testTruncateWithSnapshotNoDivergence() { INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION); - addSnapshotFeature(file, file.getContiguousBlocks()); + addSnapshotFeature(file, file.getBlocks()); // case 4: truncate to 1.5 blocks // all the blocks are in snapshot. truncate need to allocate a new block @@ -152,7 +152,7 @@ private BlockInfoContiguous newBlock(long size, short replication) { return new BlockInfoContiguous(b, replication); } - private static void addSnapshotFeature(INodeFile file, BlockInfoContiguous[] blocks) { + private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) { FileDiff diff = mock(FileDiff.class); when(diff.getBlocks()).thenReturn(blocks); FileDiffList diffList = new FileDiffList(); From 4fdd9abd7e43a0fb7b569982954a8f9660b9268b Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 15 Jul 2015 20:13:04 -0700 Subject: [PATCH 180/212] HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC to be consistent with trunk. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../server/blockmanagement/BlockInfo.java | 4 ++-- .../blockmanagement/BlockInfoContiguous.java | 10 ++++---- .../blockmanagement/BlockInfoStriped.java | 8 +++---- ...BlockInfoUnderConstructionContiguous.java} | 10 ++++---- ...=> BlockInfoUnderConstructionStriped.java} | 10 ++++---- .../server/blockmanagement/BlockManager.java | 20 ++++++++-------- .../server/namenode/FSDirWriteFileOp.java | 8 +++---- .../hdfs/server/namenode/FSEditLogLoader.java | 13 +++++----- .../hdfs/server/namenode/FSImageFormat.java | 4 ++-- .../server/namenode/FSImageFormatPBINode.java | 8 +++---- .../server/namenode/FSImageSerialization.java | 4 ++-- .../hdfs/server/namenode/FSNamesystem.java | 10 ++++---- .../FileUnderConstructionFeature.java | 3 --- .../namenode/snapshot/FileDiffList.java | 4 ++-- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 8 +++---- .../TestBlockInfoUnderConstruction.java | 3 +-- .../blockmanagement/TestBlockManager.java | 6 ++--- .../TestHeartbeatHandling.java | 7 +++--- .../TestReplicationPolicy.java | 4 ++-- .../server/namenode/TestAddStripedBlocks.java | 22 ++++++++--------- .../namenode/TestBlockUnderConstruction.java | 4 ++-- .../TestCommitBlockSynchronization.java | 4 ++-- .../server/namenode/TestFileTruncate.java | 6 ++--- .../server/namenode/TestStripedINodeFile.java | 24 +++++++++---------- .../namenode/ha/TestRetryCacheWithHA.java | 6 ++--- .../namenode/snapshot/SnapshotTestHelper.java | 4 ++-- 27 files changed, 107 insertions(+), 110 deletions(-) rename hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/{BlockInfoContiguousUnderConstruction.java => BlockInfoUnderConstructionContiguous.java} (96%) rename hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/{BlockInfoStripedUnderConstruction.java => BlockInfoUnderConstructionStriped.java} (96%) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index e2ccd9ba8640e..b135c0837e9ad 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -359,3 +359,6 @@ HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz) + + HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC + to be consistent with trunk. (zhz) 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 2b823f51cc1de..43082780e73dc 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 @@ -304,8 +304,8 @@ public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage, /** * BlockInfo represents a block that is not being constructed. * In order to start modifying the block, the BlockInfo should be converted to - * {@link BlockInfoContiguousUnderConstruction} or - * {@link BlockInfoStripedUnderConstruction}. + * {@link BlockInfoUnderConstructionContiguous} or + * {@link BlockInfoUnderConstructionStriped}. * @return {@link HdfsServerConstants.BlockUCState#COMPLETE} */ public HdfsServerConstants.BlockUCState getBlockUCState() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index bb9bf5b27bd8a..d9adccc98708e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -127,18 +127,18 @@ void replaceBlock(BlockInfo newBlock) { * Convert a complete block to an under construction block. * @return BlockInfoUnderConstruction - an under construction block. */ - public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction( + public BlockInfoUnderConstructionContiguous convertToBlockUnderConstruction( BlockUCState s, DatanodeStorageInfo[] targets) { if(isComplete()) { - BlockInfoContiguousUnderConstruction ucBlock = - new BlockInfoContiguousUnderConstruction(this, + BlockInfoUnderConstructionContiguous ucBlock = + new BlockInfoUnderConstructionContiguous(this, getBlockCollection().getPreferredBlockReplication(), s, targets); ucBlock.setBlockCollection(getBlockCollection()); return ucBlock; } // the block is already under construction - BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction) this; + BlockInfoUnderConstructionContiguous ucBlock = + (BlockInfoUnderConstructionContiguous) this; ucBlock.setBlockUCState(s); ucBlock.setExpectedLocations(targets); ucBlock.setBlockCollection(getBlockCollection()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 66745101c084a..f101dd01ba8f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -256,16 +256,16 @@ public int numNodes() { * Convert a complete block to an under construction block. * @return BlockInfoUnderConstruction - an under construction block. */ - public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( + public BlockInfoUnderConstructionStriped convertToBlockUnderConstruction( BlockUCState s, DatanodeStorageInfo[] targets) { - final BlockInfoStripedUnderConstruction ucBlock; + final BlockInfoUnderConstructionStriped ucBlock; if(isComplete()) { - ucBlock = new BlockInfoStripedUnderConstruction(this, schema, cellSize, + ucBlock = new BlockInfoUnderConstructionStriped(this, schema, cellSize, s, targets); ucBlock.setBlockCollection(getBlockCollection()); } else { // the block is already under construction - ucBlock = (BlockInfoStripedUnderConstruction) this; + ucBlock = (BlockInfoUnderConstructionStriped) this; ucBlock.setBlockUCState(s); ucBlock.setExpectedLocations(targets); ucBlock.setBlockCollection(getBlockCollection()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java similarity index 96% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java index ce2219a6a1535..c102a05194688 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java @@ -31,7 +31,7 @@ * Represents a block that is currently being constructed.
    * This is usually the last block of a file opened for write or append. */ -public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous +public class BlockInfoUnderConstructionContiguous extends BlockInfoContiguous implements BlockInfoUnderConstruction{ /** Block state. See {@link BlockUCState} */ private BlockUCState blockUCState; @@ -64,18 +64,18 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous * Create block and set its state to * {@link BlockUCState#UNDER_CONSTRUCTION}. */ - public BlockInfoContiguousUnderConstruction(Block blk, short replication) { + public BlockInfoUnderConstructionContiguous(Block blk, short replication) { this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null); } /** * Create a block that is currently being constructed. */ - public BlockInfoContiguousUnderConstruction(Block blk, short replication, + public BlockInfoUnderConstructionContiguous(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) { super(blk, replication); assert getBlockUCState() != BlockUCState.COMPLETE : - "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state"; + "BlockInfoUnderConstructionContiguous cannot be in COMPLETE state"; this.blockUCState = state; setExpectedLocations(targets); } @@ -179,7 +179,7 @@ public void initializeBlockRecovery(long recoveryId) { blockRecoveryId = recoveryId; if (replicas.size() == 0) { NameNode.blockStateChangeLog.warn("BLOCK*" - + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:" + + " BlockInfoUnderConstructionContiguous.initLeaseRecovery:" + " No blocks found, lease removed."); } boolean allLiveReplicasTriedAsPrimary = true; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java similarity index 96% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java index 5f78096d719ac..2746eeb26bf45 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java @@ -32,7 +32,7 @@ * Represents a striped block that is currently being constructed. * This is usually the last block of a file opened for write or append. */ -public class BlockInfoStripedUnderConstruction extends BlockInfoStriped +public class BlockInfoUnderConstructionStriped extends BlockInfoStriped implements BlockInfoUnderConstruction{ private BlockUCState blockUCState; @@ -57,7 +57,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped /** * Constructor with null storage targets. */ - public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, + public BlockInfoUnderConstructionStriped(Block blk, ECSchema schema, int cellSize) { this(blk, schema, cellSize, UNDER_CONSTRUCTION, null); } @@ -65,11 +65,11 @@ public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, /** * Create a striped block that is currently being constructed. */ - public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, + public BlockInfoUnderConstructionStriped(Block blk, ECSchema schema, int cellSize, BlockUCState state, DatanodeStorageInfo[] targets) { super(blk, schema, cellSize); assert getBlockUCState() != COMPLETE : - "BlockInfoStripedUnderConstruction cannot be in COMPLETE state"; + "BlockInfoUnderConstructionStriped cannot be in COMPLETE state"; this.blockUCState = state; setExpectedLocations(targets); } @@ -188,7 +188,7 @@ public void initializeBlockRecovery(long recoveryId) { blockRecoveryId = recoveryId; if (replicas == null || replicas.length == 0) { NameNode.blockStateChangeLog.warn("BLOCK*" + - " BlockInfoStripedUnderConstruction.initLeaseRecovery:" + + " BlockInfoUnderConstructionStriped.initLeaseRecovery:" + " No blocks found, lease removed."); // sets primary node index and return. primaryNodeIndex = -1; 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 57904dfad8019..edcc14e92d046 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 @@ -872,17 +872,17 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos, private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) { if (!blk.isComplete()) { if (blk.isStriped()) { - final BlockInfoStripedUnderConstruction uc = - (BlockInfoStripedUnderConstruction) blk; + final BlockInfoUnderConstructionStriped uc = + (BlockInfoUnderConstructionStriped) blk; final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, false); } else { - assert blk instanceof BlockInfoContiguousUnderConstruction; - final BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction) blk; + assert blk instanceof BlockInfoUnderConstructionContiguous; + final BlockInfoUnderConstructionContiguous uc = + (BlockInfoUnderConstructionContiguous) blk; final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); @@ -1863,8 +1863,8 @@ static class StatefulBlockInfo { StatefulBlockInfo(BlockInfo storedBlock, Block reportedBlock, ReplicaState reportedState) { Preconditions.checkArgument( - storedBlock instanceof BlockInfoContiguousUnderConstruction || - storedBlock instanceof BlockInfoStripedUnderConstruction); + storedBlock instanceof BlockInfoUnderConstructionContiguous || + storedBlock instanceof BlockInfoUnderConstructionStriped); this.storedBlock = storedBlock; this.reportedBlock = reportedBlock; this.reportedState = reportedState; @@ -2692,8 +2692,8 @@ private Block addStoredBlock(final BlockInfo block, assert block != null && namesystem.hasWriteLock(); BlockInfo storedBlock; DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); - if (block instanceof BlockInfoContiguousUnderConstruction || - block instanceof BlockInfoStripedUnderConstruction) { + if (block instanceof BlockInfoUnderConstructionContiguous || + block instanceof BlockInfoUnderConstructionStriped) { //refresh our copy in case the block got completed in another thread storedBlock = getStoredBlock(block); } else { @@ -4118,7 +4118,7 @@ public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, final LocatedBlock lb; if (info.isStriped()) { lb = newLocatedStripedBlock(eb, locs, - ((BlockInfoStripedUnderConstruction)info).getBlockIndices(), + ((BlockInfoUnderConstructionStriped)info).getBlockIndices(), offset, false); } else { lb = newLocatedBlock(eb, locs, offset, false); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 1d2b23c1e6984..254e850e41bda 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -44,8 +44,8 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -540,7 +540,7 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, // check quota limits and updated space consumed fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), numLocations, true); - blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema, + blockInfo = new BlockInfoUnderConstructionStriped(block, ecSchema, ecZone.getCellSize(), HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } else { @@ -549,7 +549,7 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, fileINode.getPreferredBlockReplication(), true); short numLocations = fileINode.getFileReplication(); - blockInfo = new BlockInfoContiguousUnderConstruction(block, + blockInfo = new BlockInfoUnderConstructionContiguous(block, numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } 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 76ae561e4a43f..a1f38a35c9878 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 @@ -42,14 +42,14 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.protocol.Block; 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.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; @@ -105,7 +105,6 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter; import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step; import org.apache.hadoop.hdfs.util.Holder; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.ChunkedArrayList; import com.google.common.base.Joiner; @@ -992,10 +991,10 @@ private void addNewBlock(AddBlockOp op, INodeFile file, final BlockInfo newBlockInfo; boolean isStriped = ecZone != null; if (isStriped) { - newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, + newBlockInfo = new BlockInfoUnderConstructionStriped(newBlock, ecZone.getSchema(), ecZone.getCellSize()); } else { - newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, + newBlockInfo = new BlockInfoUnderConstructionContiguous(newBlock, file.getPreferredBlockReplication()); } fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file); @@ -1078,10 +1077,10 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // what about an old-version fsync() where fsync isn't called // until several blocks in? if (isStriped) { - newBI = new BlockInfoStripedUnderConstruction(newBlock, + newBI = new BlockInfoUnderConstructionStriped(newBlock, ecZone.getSchema(), ecZone.getCellSize()); } else { - newBI = new BlockInfoContiguousUnderConstruction(newBlock, + newBI = new BlockInfoUnderConstructionContiguous(newBlock, file.getPreferredBlockReplication()); } } else { 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 876427c3faa34..2e490e7d3043b 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 @@ -55,7 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; @@ -780,7 +780,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, if (blocks.length > 0) { Block lastBlk = blocks[blocks.length - 1]; blocks[blocks.length - 1] = - new BlockInfoContiguousUnderConstruction(lastBlk, replication); + new BlockInfoUnderConstructionContiguous(lastBlk, replication); } } } 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 013445e59d9cc..3efb9338de396 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 @@ -44,9 +44,9 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext; @@ -373,10 +373,10 @@ private INodeFile loadINodeFile(INodeSection.INode n) { final BlockInfo ucBlk; if (isStriped) { BlockInfoStriped striped = (BlockInfoStriped) lastBlk; - ucBlk = new BlockInfoStripedUnderConstruction(striped, + ucBlk = new BlockInfoUnderConstructionStriped(striped, schema, (int)f.getStripingCellSize()); } else { - ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, + ucBlk = new BlockInfoUnderConstructionContiguous(lastBlk, replication); } file.setBlock(file.numBlocks() - 1, ucBlk); 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 af3f8139d13d4..d87378c58d228 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 @@ -34,7 +34,7 @@ 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.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap; @@ -138,7 +138,7 @@ static INodeFile readINodeUnderConstruction( // last block is UNDER_CONSTRUCTION if(numBlocks > 0) { blk.readFields(in); - blocksContiguous[i] = new BlockInfoContiguousUnderConstruction( + blocksContiguous[i] = new BlockInfoUnderConstructionContiguous( blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); } 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 50ad2517807d7..541bfa4626ed0 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 @@ -207,7 +207,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -2041,7 +2041,7 @@ boolean truncateInternal(String src, long newLength, final BlockInfo last = file.getLastBlock(); if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { final Block truncateBlock - = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock(); + = ((BlockInfoUnderConstructionContiguous)last).getTruncateBlock(); if (truncateBlock != null) { final long truncateLength = file.computeFileSize(false, false) + truncateBlock.getNumBytes(); @@ -2124,11 +2124,11 @@ Block prepareFileForTruncate(INodesInPath iip, nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock))); } - BlockInfoContiguousUnderConstruction truncatedBlockUC; + BlockInfoUnderConstructionContiguous truncatedBlockUC; if(shouldCopyOnTruncate) { // Add new truncateBlock into blocksMap and // use oldBlock as a source for copy-on-truncate recovery - truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock, + truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock, file.getPreferredBlockReplication()); truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta); truncatedBlockUC.setTruncateBlock(oldBlock); @@ -2145,7 +2145,7 @@ Block prepareFileForTruncate(INodesInPath iip, blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta); oldBlock = file.getLastBlock(); assert !oldBlock.isComplete() : "oldBlock should be under construction"; - truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock; + truncatedBlockUC = (BlockInfoUnderConstructionContiguous) oldBlock; truncatedBlockUC.setTruncateBlock(new Block(oldBlock)); truncatedBlockUC.getTruncateBlock().setNumBytes( oldBlock.getNumBytes() - lastBlockDelta); 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 a8e2e00e5105c..900839a6ad25e 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 @@ -21,9 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; /** 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 442e127cdc52b..f44ea7639bc59 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 @@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; @@ -136,7 +136,7 @@ void combineAndCollectSnapshotBlocks( Block dontRemoveBlock = null; if (lastBlock != null && lastBlock.getBlockUCState().equals( HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) { - dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock) + dontRemoveBlock = ((BlockInfoUnderConstructionContiguous) lastBlock) .getTruncateBlock(); } // Collect the remaining blocks of the file, ignoring truncate block 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 7c9eabf0d36b0..0258591d2a003 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 @@ -116,7 +116,7 @@ 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.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -1626,9 +1626,9 @@ public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn, BlockInfo storedBlock = fsn.getStoredBlock(blk.getLocalBlock()); assertTrue("Block " + blk + " should be under construction, " + "got: " + storedBlock, - storedBlock instanceof BlockInfoContiguousUnderConstruction); - BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction)storedBlock; + storedBlock instanceof BlockInfoUnderConstructionContiguous); + BlockInfoUnderConstructionContiguous ucBlock = + (BlockInfoUnderConstructionContiguous)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. final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations(); 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 f5a9cc415822c..7f080ba08bcc6 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 @@ -23,7 +23,6 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; -import org.apache.hadoop.util.Time; import org.junit.Test; /** @@ -40,7 +39,7 @@ public void testInitializeBlockRecovery() throws Exception { DatanodeDescriptor dd3 = s3.getDatanodeDescriptor(); dd1.isAlive = dd2.isAlive = dd3.isAlive = true; - BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( + BlockInfoUnderConstructionContiguous blockInfo = new BlockInfoUnderConstructionContiguous( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, BlockUCState.UNDER_CONSTRUCTION, 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 074be16e20ba2..72a6c001c17c2 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 @@ -727,7 +727,7 @@ public void testSafeModeIBRBeforeFirstFullBR() throws Exception { // verify the storage info is correct assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo (ds) >= 0); - assertTrue(((BlockInfoContiguousUnderConstruction) bm. + assertTrue(((BlockInfoUnderConstructionContiguous) bm. getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0); assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId)) .findStorageInfo(ds) >= 0); @@ -748,8 +748,8 @@ private BlockInfoContiguous addBlockToBM(long blkId) { private BlockInfoContiguous addUcBlockToBM(long blkId) { Block block = new Block(blkId); - BlockInfoContiguousUnderConstruction blockInfo = - new BlockInfoContiguousUnderConstruction(block, (short) 3); + BlockInfoUnderConstructionContiguous blockInfo = + new BlockInfoUnderConstructionContiguous(block, (short) 3); BlockCollection bc = Mockito.mock(BlockCollection.class); Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication(); bm.blocksMap.addBlockCollection(blockInfo, bc); 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 6fc30ba6d6314..4e90ddaaf3529 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 @@ -39,7 +39,6 @@ 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.util.Time; import org.junit.Test; /** @@ -173,7 +172,7 @@ public void testHeartbeatBlockRecovery() throws Exception { dd1.getStorageInfos()[0], dd2.getStorageInfos()[0], dd3.getStorageInfos()[0]}; - BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( + BlockInfoUnderConstructionContiguous blockInfo = new BlockInfoUnderConstructionContiguous( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, BlockUCState.UNDER_RECOVERY, storages); dd1.addBlockToBeRecovered(blockInfo); @@ -195,7 +194,7 @@ public void testHeartbeatBlockRecovery() throws Exception { // More than the default stale interval of 30 seconds. DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0); - blockInfo = new BlockInfoContiguousUnderConstruction( + blockInfo = new BlockInfoUnderConstructionContiguous( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, BlockUCState.UNDER_RECOVERY, storages); dd1.addBlockToBeRecovered(blockInfo); @@ -216,7 +215,7 @@ public void testHeartbeatBlockRecovery() throws Exception { // More than the default stale interval of 30 seconds. DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000); - blockInfo = new BlockInfoContiguousUnderConstruction( + blockInfo = new BlockInfoUnderConstructionContiguous( new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, BlockUCState.UNDER_RECOVERY, storages); dd1.addBlockToBeRecovered(blockInfo); 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 e699e7f14d6dc..3e282365e1f59 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 @@ -1182,7 +1182,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() // block under construction, the BlockManager will realize the expected // replication has been achieved and remove it from the under-replicated // queue. - BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1); + BlockInfoUnderConstructionContiguous info = new BlockInfoUnderConstructionContiguous(block1, (short) 1); BlockCollection bc = mock(BlockCollection.class); when(bc.getPreferredBlockReplication()).thenReturn((short)1); bm.addBlockCollection(info, bc); @@ -1247,7 +1247,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo( dataNodes[0], new DatanodeStorage("s1"))}; - final BlockInfoContiguousUnderConstruction ucBlock = + final BlockInfoUnderConstructionContiguous ucBlock = info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, storageAry); DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index b8aac71d25d7b..dd7086bd14075 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -170,8 +170,8 @@ private void checkStripedBlockUC(BlockInfoStriped block, Assert.assertEquals(0, block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); - final BlockInfoStripedUnderConstruction blockUC = - (BlockInfoStripedUnderConstruction) block; + final BlockInfoUnderConstructionStriped blockUC = + (BlockInfoUnderConstructionStriped) block; Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, blockUC.getBlockUCState()); if (checkReplica) { @@ -205,8 +205,8 @@ public void testGetLocatedStripedBlocks() throws Exception { FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); - BlockInfoStripedUnderConstruction lastBlk = - (BlockInfoStripedUnderConstruction) fileNode.getLastBlock(); + BlockInfoUnderConstructionStriped lastBlk = + (BlockInfoUnderConstructionStriped) fileNode.getLastBlock(); DatanodeInfo[] expectedDNs = DatanodeStorageInfo .toDatanodeInfos(lastBlk.getExpectedStorageLocations()); int[] indices = lastBlk.getBlockIndices(); @@ -228,7 +228,7 @@ public void testGetLocatedStripedBlocks() throws Exception { } /** - * Test BlockInfoStripedUnderConstruction#addReplicaIfNotPresent in different + * Test BlockInfoUnderConstructionStriped#addReplicaIfNotPresent in different * scenarios. */ @Test @@ -246,8 +246,8 @@ public void testAddUCReplica() throws Exception { cluster.getNamesystem().getAdditionalBlock(file.toString(), fileNode.getId(), dfs.getClient().getClientName(), null, null, null); BlockInfo lastBlock = fileNode.getLastBlock(); - BlockInfoStripedUnderConstruction ucBlock = - (BlockInfoStripedUnderConstruction) lastBlock; + BlockInfoUnderConstructionStriped ucBlock = + (BlockInfoUnderConstructionStriped) lastBlock; DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); int[] indices = ucBlock.getBlockIndices(); @@ -255,7 +255,7 @@ public void testAddUCReplica() throws Exception { Assert.assertEquals(GROUP_SIZE, indices.length); // 2. mimic incremental block reports and make sure the uc-replica list in - // the BlockStripedUC is correct + // the BlockInfoUCStriped is correct int i = 0; for (DataNode dn : cluster.getDataNodes()) { final Block block = new Block(lastBlock.getBlockId() + i++, @@ -307,8 +307,8 @@ public void testAddUCReplica() throws Exception { bpId, reports, null); } - BlockInfoStripedUnderConstruction ucBlock = - (BlockInfoStripedUnderConstruction) lastBlock; + BlockInfoUnderConstructionStriped ucBlock = + (BlockInfoUnderConstructionStriped) lastBlock; DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); int[] indices = ucBlock.getBlockIndices(); Assert.assertEquals(GROUP_SIZE, locs.length); 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 f372bec3ba62e..4d5c4de44c3f7 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 @@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.junit.AfterClass; @@ -170,7 +170,7 @@ public void testGetBlockLocations() throws IOException { final List blocks = lb.getLocatedBlocks(); assertEquals(i, blocks.size()); final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock(); - assertTrue(b instanceof BlockInfoContiguousUnderConstruction); + assertTrue(b instanceof BlockInfoUnderConstructionContiguous); if (++i < NUM_BLOCKS) { // write one more block 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 ea560feededae..e1b1cd332755e 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 @@ -23,7 +23,7 @@ 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.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.junit.Test; @@ -67,7 +67,7 @@ private FSNamesystem makeNameSystemSpy(Block block, INodeFile file) namesystem.dir.getINodeMap().put(file); FSNamesystem namesystemSpy = spy(namesystem); - BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( + BlockInfoUnderConstructionContiguous blockInfo = new BlockInfoUnderConstructionContiguous( block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); blockInfo.setBlockCollection(file); blockInfo.setGenerationStamp(genStamp); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java index ffa1451d02472..53a5d67f2272c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java @@ -55,7 +55,7 @@ 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.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.security.UserGroupInformation; @@ -1020,7 +1020,7 @@ public void testTruncateRecovery() throws IOException { is(fsn.getBlockIdManager().getGenerationStampV2())); assertThat(file.getLastBlock().getBlockUCState(), is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY)); - long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock()) + long blockRecoveryId = ((BlockInfoUnderConstructionContiguous) file.getLastBlock()) .getBlockRecoveryId(); assertThat(blockRecoveryId, is(initialGenStamp + 1)); fsn.getEditLog().logTruncate( @@ -1054,7 +1054,7 @@ public void testTruncateRecovery() throws IOException { is(fsn.getBlockIdManager().getGenerationStampV2())); assertThat(file.getLastBlock().getBlockUCState(), is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY)); - long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock()) + long blockRecoveryId = ((BlockInfoUnderConstructionContiguous) file.getLastBlock()) .getBlockRecoveryId(); assertThat(blockRecoveryId, is(initialGenStamp + 1)); fsn.getEditLog().logTruncate( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 477c609936f9a..9cff6144f6e71 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.io.erasurecode.ECSchema; @@ -146,14 +146,14 @@ public void testBlockStripedFileSize() } @Test - public void testBlockStripedUCFileSize() + public void testBlockUCStripedFileSize() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); - BlockInfoStripedUnderConstruction bInfoStripedUC - = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); - bInfoStripedUC.setNumBytes(100); - inf.addBlock(bInfoStripedUC); + BlockInfoUnderConstructionStriped bInfoUCStriped + = new BlockInfoUnderConstructionStriped(blk, testSchema, cellSize); + bInfoUCStriped.setNumBytes(100); + inf.addBlock(bInfoUCStriped); assertEquals(100, inf.computeFileSize()); assertEquals(0, inf.computeFileSize(false, false)); } @@ -180,21 +180,21 @@ public void testBlockStripedComputeQuotaUsage() } @Test - public void testBlockStripedUCComputeQuotaUsage() + public void testBlockUCStripedComputeQuotaUsage() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); - BlockInfoStripedUnderConstruction bInfoStripedUC - = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); - bInfoStripedUC.setNumBytes(100); - inf.addBlock(bInfoStripedUC); + BlockInfoUnderConstructionStriped bInfoUCStriped + = new BlockInfoUnderConstructionStriped(blk, testSchema, cellSize); + bInfoUCStriped.setNumBytes(100); + inf.addBlock(bInfoUCStriped); QuotaCounts counts = inf.computeQuotaUsageWithStriped(defaultPolicy, new QuotaCounts.Builder().build()); assertEquals(1024, inf.getPreferredBlockSize()); assertEquals(1, counts.getNameSpace()); - // Consumed space in the case of BlockInfoStripedUC can be calculated + // Consumed space in the case of BlockInfoUCStriped can be calculated // by using preferred block size. This is 1024 and total block num // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216. assertEquals(9216, counts.getStorageSpace()); 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 e3572abde50a9..69694ad364658 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 @@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper; @@ -744,8 +744,8 @@ void invoke() throws Exception { boolean checkNamenodeBeforeReturn() throws Exception { INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory() .getINode4Write(file).asFile(); - BlockInfoContiguousUnderConstruction blkUC = - (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1]; + BlockInfoUnderConstructionContiguous blkUC = + (BlockInfoUnderConstructionContiguous) (fileNode.getBlocks())[1]; int datanodeNum = blkUC.getExpectedStorageLocations().length; for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) { Thread.sleep(1000); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java index 11b19f3af2c47..38d0daa180162 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java @@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage; import org.apache.hadoop.hdfs.server.datanode.BlockScanner; @@ -177,7 +177,7 @@ public static void checkSnapshotCreation(DistributedFileSystem hdfs, * Specific information for different types of INode: * {@link INodeDirectory}:childrenSize * {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()} - * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information. + * and {@link BlockInfoUnderConstructionContiguous#toString()} for detailed information. * {@link FileWithSnapshot}: next link * * @see INode#dumpTreeRecursively() From 06394e37601186d2bcff49ccea00712fda9b3579 Mon Sep 17 00:00:00 2001 From: Walter Su Date: Mon, 20 Jul 2015 10:18:34 +0800 Subject: [PATCH 181/212] HDFS-8433. Erasure coding: set blockToken in LocatedStripedBlock. Contributed by Walter Su. --- .../hdfs/protocol/LocatedStripedBlock.java | 16 + .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../hadoop/hdfs/protocolPB/PBHelper.java | 15 +- .../server/blockmanagement/BlockManager.java | 21 +- .../hadoop/hdfs/util/StripedBlockUtil.java | 13 +- .../hadoop-hdfs/src/main/proto/hdfs.proto | 3 + .../TestBlockTokenWithDFS.java | 422 ++++++++++-------- .../TestBlockTokenWithDFSStriped.java | 119 +++++ 8 files changed, 407 insertions(+), 204 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java index dc5a77fab2271..6e62220ac9508 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -20,6 +20,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.security.token.Token; import java.util.Arrays; @@ -32,8 +34,10 @@ @InterfaceStability.Evolving public class LocatedStripedBlock extends LocatedBlock { private static final int[] EMPTY_INDICES = {}; + private static final Token EMPTY_TOKEN = new Token<>(); private int[] blockIndices; + private Token[] blockTokens; public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs, StorageType[] storageTypes, int[] indices, @@ -46,6 +50,10 @@ public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs, this.blockIndices = new int[indices.length]; System.arraycopy(indices, 0, blockIndices, 0, indices.length); } + blockTokens = new Token[blockIndices.length]; + for (int i = 0; i < blockIndices.length; i++) { + blockTokens[i] = EMPTY_TOKEN; + } } @Override @@ -67,4 +75,12 @@ public int[] getBlockIndices() { public boolean isStriped() { return true; } + + public Token[] getBlockTokens() { + return blockTokens; + } + + public void setBlockTokens(Token[] tokens) { + this.blockTokens = tokens; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index b135c0837e9ad..4709388662849 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -362,3 +362,5 @@ HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC to be consistent with trunk. (zhz) + + HDFS-8433. Erasure coding: set blockToken in LocatedStripedBlock.(waltersu4549) 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 6baa005c6792f..1bc096498e8da 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 @@ -813,9 +813,12 @@ public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) { builder.addAllStorageIDs(Arrays.asList(storageIDs)); } if (b instanceof LocatedStripedBlock) { - int[] indices = ((LocatedStripedBlock) b).getBlockIndices(); - for (int index : indices) { - builder.addBlockIndex(index); + LocatedStripedBlock sb = (LocatedStripedBlock) b; + int[] indices = sb.getBlockIndices(); + Token[] blockTokens = sb.getBlockTokens(); + for (int i = 0; i < indices.length; i++) { + builder.addBlockIndex(indices[i]); + builder.addBlockTokens(PBHelper.convert(blockTokens[i])); } } @@ -872,6 +875,12 @@ public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) { storageIDs, storageTypes, indices, proto.getOffset(), proto.getCorrupt(), cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()])); + List tokenProtos = proto.getBlockTokensList(); + Token[] blockTokens = new Token[indices.length]; + for (int i = 0; i < indices.length; i++) { + blockTokens[i] = PBHelper.convert(tokenProtos.get(i)); + } + ((LocatedStripedBlock) lb).setBlockTokens(blockTokens); } lb.setBlockToken(PBHelper.convert(proto.getBlockToken())); 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 edcc14e92d046..7872baa22e583 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 @@ -92,6 +92,7 @@ import org.apache.hadoop.net.Node; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.LightWeightGSet; import org.apache.hadoop.util.Time; @@ -989,9 +990,23 @@ public void setBlockToken(final LocatedBlock b, final AccessMode mode) throws IOException { if (isBlockTokenEnabled()) { // Use cached UGI if serving RPC calls. - b.setBlockToken(blockTokenSecretManager.generateToken( - NameNode.getRemoteUser().getShortUserName(), - b.getBlock(), EnumSet.of(mode))); + if (b.isStriped()) { + LocatedStripedBlock sb = (LocatedStripedBlock) b; + int[] indices = sb.getBlockIndices(); + Token[] blockTokens = new Token[indices.length]; + ExtendedBlock internalBlock = new ExtendedBlock(b.getBlock()); + for (int i = 0; i < indices.length; i++) { + internalBlock.setBlockId(b.getBlock().getBlockId() + indices[i]); + blockTokens[i] = blockTokenSecretManager.generateToken( + NameNode.getRemoteUser().getShortUserName(), + internalBlock, EnumSet.of(mode)); + } + sb.setBlockTokens(blockTokens); + } else { + b.setBlockToken(blockTokenSecretManager.generateToken( + NameNode.getRemoteUser().getShortUserName(), + b.getBlock(), EnumSet.of(mode))); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 6bd5e1f01030f..9b0939c6dcca4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -30,8 +30,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import com.google.common.base.Preconditions; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.security.token.Token; import java.nio.ByteBuffer; import java.util.*; @@ -105,17 +107,22 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, int idxInBlockGroup) { final ExtendedBlock blk = constructInternalBlock( bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); - + final LocatedBlock locatedBlock; if (idxInReturnedLocs < bg.getLocations().length) { - return new LocatedBlock(blk, + locatedBlock = new LocatedBlock(blk, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, bg.getStartOffset(), bg.isCorrupt(), null); } else { - return new LocatedBlock(blk, null, null, null, + locatedBlock = new LocatedBlock(blk, null, null, null, bg.getStartOffset(), bg.isCorrupt(), null); } + Token[] blockTokens = bg.getBlockTokens(); + if (idxInBlockGroup < blockTokens.length) { + locatedBlock.setBlockToken(blockTokens[idxInBlockGroup]); + } + return locatedBlock; } /** 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 e1f944fd72567..d2cb66538efbd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -220,7 +220,10 @@ message LocatedBlockProto { repeated bool isCached = 6 [packed=true]; // if a location in locs is cached repeated StorageTypeProto storageTypes = 7; repeated string storageIDs = 8; + + // striped block related fields repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage + repeated hadoop.common.TokenProto blockTokens = 10; // each internal block has a block token } message DataEncryptionKeyProto { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java index 43f299234322c..26ed1fe56399c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; -import java.util.EnumSet; import java.util.List; import java.util.Random; @@ -69,28 +68,32 @@ public class TestBlockTokenWithDFS { - private static final int BLOCK_SIZE = 1024; - private static final int FILE_SIZE = 2 * BLOCK_SIZE; + protected static int BLOCK_SIZE = 1024; + protected static int FILE_SIZE = 2 * BLOCK_SIZE; private static final String FILE_TO_READ = "/fileToRead.dat"; private static final String FILE_TO_WRITE = "/fileToWrite.dat"; private static final String FILE_TO_APPEND = "/fileToAppend.dat"; - private final byte[] rawData = new byte[FILE_SIZE]; { ((Log4JLogger) DFSClient.LOG).getLogger().setLevel(Level.ALL); + } + + public static byte[] generateBytes(int fileSize){ Random r = new Random(); + byte[] rawData = new byte[fileSize]; r.nextBytes(rawData); + return rawData; } - private void createFile(FileSystem fs, Path filename) throws IOException { + private void createFile(FileSystem fs, Path filename, byte[] expected) throws IOException { FSDataOutputStream out = fs.create(filename); - out.write(rawData); + out.write(expected); out.close(); } // read a file using blockSeekTo() - private boolean checkFile1(FSDataInputStream in) { - byte[] toRead = new byte[FILE_SIZE]; + private boolean checkFile1(FSDataInputStream in, byte[] expected) { + byte[] toRead = new byte[expected.length]; int totalRead = 0; int nRead = 0; try { @@ -101,27 +104,27 @@ private boolean checkFile1(FSDataInputStream in) { return false; } assertEquals("Cannot read file.", toRead.length, totalRead); - return checkFile(toRead); + return checkFile(toRead, expected); } // read a file using fetchBlockByteRange() - private boolean checkFile2(FSDataInputStream in) { - byte[] toRead = new byte[FILE_SIZE]; + private boolean checkFile2(FSDataInputStream in, byte[] expected) { + byte[] toRead = new byte[expected.length]; try { assertEquals("Cannot read file", toRead.length, in.read(0, toRead, 0, toRead.length)); } catch (IOException e) { return false; } - return checkFile(toRead); + return checkFile(toRead, expected); } - private boolean checkFile(byte[] fileToCheck) { - if (fileToCheck.length != rawData.length) { + private boolean checkFile(byte[] fileToCheck, byte[] expected) { + if (fileToCheck.length != expected.length) { return false; } for (int i = 0; i < fileToCheck.length; i++) { - if (fileToCheck[i] != rawData[i]) { + if (fileToCheck[i] != expected[i]) { return false; } } @@ -137,7 +140,7 @@ private static FSDataOutputStream writeFile(FileSystem fileSys, Path name, } // try reading a block using a BlockReader directly - private static void tryRead(final Configuration conf, LocatedBlock lblock, + protected void tryRead(final Configuration conf, LocatedBlock lblock, boolean shouldSucceed) { InetSocketAddress targetAddr = null; IOException ioe = null; @@ -148,7 +151,7 @@ private static void tryRead(final Configuration conf, LocatedBlock lblock, targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); blockReader = new BlockReaderFactory(new DfsClientConf(conf)). - setFileName(BlockReaderFactory.getFileName(targetAddr, + setFileName(BlockReaderFactory.getFileName(targetAddr, "test-blockpoolid", block.getBlockId())). setBlock(block). setBlockToken(lblock.getBlockToken()). @@ -205,7 +208,7 @@ public Peer newConnectedPeer(InetSocketAddress addr, } // get a conf for testing - private static Configuration getConf(int numDataNodes) { + protected Configuration getConf(int numDataNodes) { Configuration conf = new Configuration(); conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); @@ -241,16 +244,16 @@ public void testAppend() throws Exception { SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); Path fileToAppend = new Path(FILE_TO_APPEND); FileSystem fs = cluster.getFileSystem(); - + byte[] expected = generateBytes(FILE_SIZE); // write a one-byte file FSDataOutputStream stm = writeFile(fs, fileToAppend, (short) numDataNodes, BLOCK_SIZE); - stm.write(rawData, 0, 1); + stm.write(expected, 0, 1); stm.close(); // open the file again for append stm = fs.append(fileToAppend); - int mid = rawData.length - 1; - stm.write(rawData, 1, mid - 1); + int mid = expected.length - 1; + stm.write(expected, 1, mid - 1); stm.hflush(); /* @@ -267,11 +270,11 @@ public void testAppend() throws Exception { // remove a datanode to force re-establishing pipeline cluster.stopDataNode(0); // append the rest of the file - stm.write(rawData, mid, rawData.length - mid); + stm.write(expected, mid, expected.length - mid); stm.close(); // check if append is successful FSDataInputStream in5 = fs.open(fileToAppend); - assertTrue(checkFile1(in5)); + assertTrue(checkFile1(in5, expected)); } finally { if (cluster != null) { cluster.shutdown(); @@ -303,11 +306,12 @@ public void testWrite() throws Exception { Path fileToWrite = new Path(FILE_TO_WRITE); FileSystem fs = cluster.getFileSystem(); + byte[] expected = generateBytes(FILE_SIZE); FSDataOutputStream stm = writeFile(fs, fileToWrite, (short) numDataNodes, BLOCK_SIZE); // write a partial block - int mid = rawData.length - 1; - stm.write(rawData, 0, mid); + int mid = expected.length - 1; + stm.write(expected, 0, mid); stm.hflush(); /* @@ -324,11 +328,11 @@ public void testWrite() throws Exception { // remove a datanode to force re-establishing pipeline cluster.stopDataNode(0); // write the rest of the file - stm.write(rawData, mid, rawData.length - mid); + stm.write(expected, mid, expected.length - mid); stm.close(); // check if write is successful FSDataInputStream in4 = fs.open(fileToWrite); - assertTrue(checkFile1(in4)); + assertTrue(checkFile1(in4, expected)); } finally { if (cluster != null) { cluster.shutdown(); @@ -346,125 +350,137 @@ public void testRead() throws Exception { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build(); cluster.waitActive(); assertEquals(numDataNodes, cluster.getDataNodes().size()); + doTestRead(conf, cluster, false); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } - final NameNode nn = cluster.getNameNode(); - final NamenodeProtocols nnProto = nn.getRpcServer(); - final BlockManager bm = nn.getNamesystem().getBlockManager(); - final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); + protected void doTestRead(Configuration conf, MiniDFSCluster cluster, + boolean isStriped) throws Exception { + final int numDataNodes = cluster.getDataNodes().size(); + final NameNode nn = cluster.getNameNode(); + final NamenodeProtocols nnProto = nn.getRpcServer(); + final BlockManager bm = nn.getNamesystem().getBlockManager(); + final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); - // set a short token lifetime (1 second) initially - SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); + // set a short token lifetime (1 second) initially + SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); - Path fileToRead = new Path(FILE_TO_READ); - FileSystem fs = cluster.getFileSystem(); - createFile(fs, fileToRead); + Path fileToRead = new Path(FILE_TO_READ); + FileSystem fs = cluster.getFileSystem(); + byte[] expected = generateBytes(FILE_SIZE); + createFile(fs, fileToRead, expected); /* * setup for testing expiration handling of cached tokens */ - // read using blockSeekTo(). Acquired tokens are cached in in1 - FSDataInputStream in1 = fs.open(fileToRead); - assertTrue(checkFile1(in1)); - // read using blockSeekTo(). Acquired tokens are cached in in2 - FSDataInputStream in2 = fs.open(fileToRead); - assertTrue(checkFile1(in2)); - // read using fetchBlockByteRange(). Acquired tokens are cached in in3 - FSDataInputStream in3 = fs.open(fileToRead); - assertTrue(checkFile2(in3)); + // read using blockSeekTo(). Acquired tokens are cached in in1 + FSDataInputStream in1 = fs.open(fileToRead); + assertTrue(checkFile1(in1,expected)); + // read using blockSeekTo(). Acquired tokens are cached in in2 + FSDataInputStream in2 = fs.open(fileToRead); + assertTrue(checkFile1(in2,expected)); + // read using fetchBlockByteRange(). Acquired tokens are cached in in3 + FSDataInputStream in3 = fs.open(fileToRead); + assertTrue(checkFile2(in3,expected)); /* * testing READ interface on DN using a BlockReader */ - DFSClient client = null; - try { - client = new DFSClient(new InetSocketAddress("localhost", + DFSClient client = null; + try { + client = new DFSClient(new InetSocketAddress("localhost", cluster.getNameNodePort()), conf); - } finally { - if (client != null) client.close(); - } - List locatedBlocks = nnProto.getBlockLocations( - FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks(); - LocatedBlock lblock = locatedBlocks.get(0); // first block - Token myToken = lblock.getBlockToken(); - // verify token is not expired - assertFalse(SecurityTestUtil.isBlockTokenExpired(myToken)); - // read with valid token, should succeed - tryRead(conf, lblock, true); + } finally { + if (client != null) client.close(); + } + List locatedBlocks = nnProto.getBlockLocations( + FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks(); + LocatedBlock lblock = locatedBlocks.get(0); // first block + // verify token is not expired + assertFalse(isBlockTokenExpired(lblock)); + // read with valid token, should succeed + tryRead(conf, lblock, true); /* * wait till myToken and all cached tokens in in1, in2 and in3 expire */ - while (!SecurityTestUtil.isBlockTokenExpired(myToken)) { - try { - Thread.sleep(10); - } catch (InterruptedException ignored) { - } + while (!isBlockTokenExpired(lblock)) { + try { + Thread.sleep(10); + } catch (InterruptedException ignored) { } + } /* * continue testing READ interface on DN using a BlockReader */ - // verify token is expired - assertTrue(SecurityTestUtil.isBlockTokenExpired(myToken)); - // read should fail - tryRead(conf, lblock, false); - // use a valid new token - lblock.setBlockToken(sm.generateToken(lblock.getBlock(), - EnumSet.of(BlockTokenIdentifier.AccessMode.READ))); - // read should succeed - tryRead(conf, lblock, true); - // use a token with wrong blockID - ExtendedBlock wrongBlock = new ExtendedBlock(lblock.getBlock() - .getBlockPoolId(), lblock.getBlock().getBlockId() + 1); - lblock.setBlockToken(sm.generateToken(wrongBlock, - EnumSet.of(BlockTokenIdentifier.AccessMode.READ))); - // read should fail - tryRead(conf, lblock, false); - // use a token with wrong access modes - lblock.setBlockToken(sm.generateToken(lblock.getBlock(), - EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE, - BlockTokenIdentifier.AccessMode.COPY, - BlockTokenIdentifier.AccessMode.REPLACE))); - // read should fail - tryRead(conf, lblock, false); - - // set a long token lifetime for future tokens - SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L); + // verify token is expired + assertTrue(isBlockTokenExpired(lblock)); + // read should fail + tryRead(conf, lblock, false); + // use a valid new token + bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ); + // read should succeed + tryRead(conf, lblock, true); + // use a token with wrong blockID + long rightId = lblock.getBlock().getBlockId(); + long wrongId = rightId + 1; + lblock.getBlock().setBlockId(wrongId); + bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ); + lblock.getBlock().setBlockId(rightId); + // read should fail + tryRead(conf, lblock, false); + // use a token with wrong access modes + bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.WRITE); + // read should fail + tryRead(conf, lblock, false); + + // set a long token lifetime for future tokens + SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L); /* * testing that when cached tokens are expired, DFSClient will re-fetch * tokens transparently for READ. */ - // confirm all tokens cached in in1 are expired by now - List lblocks = DFSTestUtil.getAllBlocks(in1); - for (LocatedBlock blk : lblocks) { - assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() is able to re-fetch token transparently - in1.seek(0); - assertTrue(checkFile1(in1)); - - // confirm all tokens cached in in2 are expired by now - List lblocks2 = DFSTestUtil.getAllBlocks(in2); - for (LocatedBlock blk : lblocks2) { - assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() is able to re-fetch token transparently (testing - // via another interface method) + // confirm all tokens cached in in1 are expired by now + List lblocks = DFSTestUtil.getAllBlocks(in1); + for (LocatedBlock blk : lblocks) { + assertTrue(isBlockTokenExpired(blk)); + } + // verify blockSeekTo() is able to re-fetch token transparently + in1.seek(0); + assertTrue(checkFile1(in1, expected)); + + // confirm all tokens cached in in2 are expired by now + List lblocks2 = DFSTestUtil.getAllBlocks(in2); + for (LocatedBlock blk : lblocks2) { + assertTrue(isBlockTokenExpired(blk)); + } + // verify blockSeekTo() is able to re-fetch token transparently (testing + // via another interface method) + if (isStriped) { + // striped block doesn't support seekToNewSource + in2.seek(0); + } else { assertTrue(in2.seekToNewSource(0)); - assertTrue(checkFile1(in2)); + } + assertTrue(checkFile1(in2,expected)); - // confirm all tokens cached in in3 are expired by now - List lblocks3 = DFSTestUtil.getAllBlocks(in3); - for (LocatedBlock blk : lblocks3) { - assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify fetchBlockByteRange() is able to re-fetch token transparently - assertTrue(checkFile2(in3)); + // confirm all tokens cached in in3 are expired by now + List lblocks3 = DFSTestUtil.getAllBlocks(in3); + for (LocatedBlock blk : lblocks3) { + assertTrue(isBlockTokenExpired(blk)); + } + // verify fetchBlockByteRange() is able to re-fetch token transparently + assertTrue(checkFile2(in3,expected)); /* * testing that after datanodes are restarted on the same ports, cached @@ -473,37 +489,42 @@ public void testRead() throws Exception { * new tokens can be fetched from namenode). */ - // restart datanodes on the same ports that they currently use - assertTrue(cluster.restartDataNodes(true)); - cluster.waitActive(); - assertEquals(numDataNodes, cluster.getDataNodes().size()); - cluster.shutdownNameNode(0); + // restart datanodes on the same ports that they currently use + assertTrue(cluster.restartDataNodes(true)); + cluster.waitActive(); + assertEquals(numDataNodes, cluster.getDataNodes().size()); + cluster.shutdownNameNode(0); - // confirm tokens cached in in1 are still valid - lblocks = DFSTestUtil.getAllBlocks(in1); - for (LocatedBlock blk : lblocks) { - assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() still works (forced to use cached tokens) - in1.seek(0); - assertTrue(checkFile1(in1)); - - // confirm tokens cached in in2 are still valid - lblocks2 = DFSTestUtil.getAllBlocks(in2); - for (LocatedBlock blk : lblocks2) { - assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() still works (forced to use cached tokens) + // confirm tokens cached in in1 are still valid + lblocks = DFSTestUtil.getAllBlocks(in1); + for (LocatedBlock blk : lblocks) { + assertFalse(isBlockTokenExpired(blk)); + } + // verify blockSeekTo() still works (forced to use cached tokens) + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + + // confirm tokens cached in in2 are still valid + lblocks2 = DFSTestUtil.getAllBlocks(in2); + for (LocatedBlock blk : lblocks2) { + assertFalse(isBlockTokenExpired(blk)); + } + + // verify blockSeekTo() still works (forced to use cached tokens) + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); + } + assertTrue(checkFile1(in2,expected)); - // confirm tokens cached in in3 are still valid - lblocks3 = DFSTestUtil.getAllBlocks(in3); - for (LocatedBlock blk : lblocks3) { - assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify fetchBlockByteRange() still works (forced to use cached tokens) - assertTrue(checkFile2(in3)); + // confirm tokens cached in in3 are still valid + lblocks3 = DFSTestUtil.getAllBlocks(in3); + for (LocatedBlock blk : lblocks3) { + assertFalse(isBlockTokenExpired(blk)); + } + // verify fetchBlockByteRange() still works (forced to use cached tokens) + assertTrue(checkFile2(in3,expected)); /* * testing that when namenode is restarted, cached tokens should still @@ -512,18 +533,23 @@ public void testRead() throws Exception { * setup for this test depends on the previous test. */ - // restart the namenode and then shut it down for test - cluster.restartNameNode(0); - cluster.shutdownNameNode(0); + // restart the namenode and then shut it down for test + cluster.restartNameNode(0); + cluster.shutdownNameNode(0); - // verify blockSeekTo() still works (forced to use cached tokens) - in1.seek(0); - assertTrue(checkFile1(in1)); - // verify again blockSeekTo() still works (forced to use cached tokens) + // verify blockSeekTo() still works (forced to use cached tokens) + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + // verify again blockSeekTo() still works (forced to use cached tokens) + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); - // verify fetchBlockByteRange() still works (forced to use cached tokens) - assertTrue(checkFile2(in3)); + } + assertTrue(checkFile1(in2,expected)); + + // verify fetchBlockByteRange() still works (forced to use cached tokens) + assertTrue(checkFile2(in3,expected)); /* * testing that after both namenode and datanodes got restarted (namenode @@ -532,58 +558,60 @@ public void testRead() throws Exception { * setup of this test depends on the previous test. */ - // restore the cluster and restart the datanodes for test - cluster.restartNameNode(0); - assertTrue(cluster.restartDataNodes(true)); - cluster.waitActive(); - assertEquals(numDataNodes, cluster.getDataNodes().size()); - - // shutdown namenode so that DFSClient can't get new tokens from namenode - cluster.shutdownNameNode(0); - - // verify blockSeekTo() fails (cached tokens become invalid) - in1.seek(0); - assertFalse(checkFile1(in1)); - // verify fetchBlockByteRange() fails (cached tokens become invalid) - assertFalse(checkFile2(in3)); - - // restart the namenode to allow DFSClient to re-fetch tokens - cluster.restartNameNode(0); - // verify blockSeekTo() works again (by transparently re-fetching - // tokens from namenode) - in1.seek(0); - assertTrue(checkFile1(in1)); + // restore the cluster and restart the datanodes for test + cluster.restartNameNode(0); + assertTrue(cluster.restartDataNodes(true)); + cluster.waitActive(); + assertEquals(numDataNodes, cluster.getDataNodes().size()); + + // shutdown namenode so that DFSClient can't get new tokens from namenode + cluster.shutdownNameNode(0); + + // verify blockSeekTo() fails (cached tokens become invalid) + in1.seek(0); + assertFalse(checkFile1(in1,expected)); + // verify fetchBlockByteRange() fails (cached tokens become invalid) + assertFalse(checkFile2(in3,expected)); + + // restart the namenode to allow DFSClient to re-fetch tokens + cluster.restartNameNode(0); + // verify blockSeekTo() works again (by transparently re-fetching + // tokens from namenode) + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); - // verify fetchBlockByteRange() works again (by transparently - // re-fetching tokens from namenode) - assertTrue(checkFile2(in3)); + } + assertTrue(checkFile1(in2,expected)); + // verify fetchBlockByteRange() works again (by transparently + // re-fetching tokens from namenode) + assertTrue(checkFile2(in3,expected)); /* * testing that when datanodes are restarted on different ports, DFSClient * is able to re-fetch tokens transparently to connect to them */ - // restart datanodes on newly assigned ports - assertTrue(cluster.restartDataNodes(false)); - cluster.waitActive(); - assertEquals(numDataNodes, cluster.getDataNodes().size()); - // verify blockSeekTo() is able to re-fetch token transparently - in1.seek(0); - assertTrue(checkFile1(in1)); - // verify blockSeekTo() is able to re-fetch token transparently + // restart datanodes on newly assigned ports + assertTrue(cluster.restartDataNodes(false)); + cluster.waitActive(); + assertEquals(numDataNodes, cluster.getDataNodes().size()); + // verify blockSeekTo() is able to re-fetch token transparently + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + // verify blockSeekTo() is able to re-fetch token transparently + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); - // verify fetchBlockByteRange() is able to re-fetch token transparently - assertTrue(checkFile2(in3)); - - } finally { - if (cluster != null) { - cluster.shutdown(); - } } - } + assertTrue(checkFile1(in2,expected)); + // verify fetchBlockByteRange() is able to re-fetch token transparently + assertTrue(checkFile2(in3,expected)); + } /** * Integration testing of access token, involving NN, DN, and Balancer */ @@ -593,4 +621,8 @@ public void testEnd2End() throws Exception { conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); new TestBalancer().integrationTest(conf); } + + protected boolean isBlockTokenExpired(LocatedBlock lb) throws IOException { + return SecurityTestUtil.isBlockTokenExpired(lb.getBlockToken()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java new file mode 100644 index 0000000000000..e212917b6f7aa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java @@ -0,0 +1,119 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS { + + private final static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private final static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + private final static int numDNs = dataBlocks + parityBlocks + 2; + private static MiniDFSCluster cluster; + private static Configuration conf; + + { + BLOCK_SIZE = cellSize * stripesPerBlock; + FILE_SIZE = BLOCK_SIZE * dataBlocks * 3; + } + + @Before + public void setup() throws IOException { + conf = getConf(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient() + .createErasureCodingZone("/", null, cellSize); + cluster.waitActive(); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + private Configuration getConf() { + Configuration conf = super.getConf(numDNs); + conf.setInt("io.bytes.per.checksum", cellSize); + return conf; + } + + @Test + @Override + public void testRead() throws Exception { + //TODO: DFSStripedInputStream handles token expiration +// doTestRead(conf, cluster, true); + } + + @Test + @Override + public void testWrite() throws Exception { + //TODO: DFSStripedOutputStream handles token expiration + } + + @Test + @Override + public void testAppend() throws Exception { + //TODO: support Append for striped file + } + + @Test + @Override + public void testEnd2End() throws Exception { + //TODO: DFSStripedOutputStream handles token expiration + } + + @Override + protected void tryRead(final Configuration conf, LocatedBlock lblock, + boolean shouldSucceed) { + LocatedStripedBlock lsb = (LocatedStripedBlock) lblock; + LocatedBlock[] internalBlocks = StripedBlockUtil.parseStripedBlockGroup + (lsb, cellSize, dataBlocks, parityBlocks); + for (LocatedBlock internalBlock : internalBlocks) { + super.tryRead(conf, internalBlock, shouldSucceed); + } + } + + @Override + protected boolean isBlockTokenExpired(LocatedBlock lb) throws IOException { + LocatedStripedBlock lsb = (LocatedStripedBlock) lb; + LocatedBlock[] internalBlocks = StripedBlockUtil.parseStripedBlockGroup + (lsb, cellSize, dataBlocks, parityBlocks); + for (LocatedBlock internalBlock : internalBlocks) { + if(super.isBlockTokenExpired(internalBlock)){ + return true; + } + } + return false; + } +} From 29495cb8f6b940caa9964c39a290ef233ce1ec7c Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 20 Jul 2015 10:15:14 -0700 Subject: [PATCH 182/212] HADOOP-12060. Fix ByteBuffer usage for raw erasure coders. Contributed by Kai Zheng. --- .../hadoop-common/CHANGES-HDFS-EC-7285.txt | 5 +- .../hadoop/io/erasurecode/CodecUtil.java | 38 ++++---- .../apache/hadoop/io/erasurecode/ECBlock.java | 14 +-- .../hadoop/io/erasurecode/ECBlockGroup.java | 6 +- .../apache/hadoop/io/erasurecode/ECChunk.java | 6 +- .../hadoop/io/erasurecode/ECSchema.java | 18 ++-- .../hadoop/io/erasurecode/SchemaLoader.java | 3 +- .../rawcoder/AbstractRawErasureCoder.java | 42 +++++---- .../rawcoder/AbstractRawErasureDecoder.java | 50 +++++----- .../rawcoder/AbstractRawErasureEncoder.java | 27 +++--- .../rawcoder/RawErasureCoderFactory.java | 8 +- .../rawcoder/RawErasureDecoder.java | 25 +++-- .../rawcoder/RawErasureEncoder.java | 24 +++-- .../io/erasurecode/BufferAllocator.java | 91 +++++++++++++++++++ .../hadoop/io/erasurecode/TestCoderBase.java | 17 +++- .../coder/TestErasureCoderBase.java | 10 +- .../rawcoder/TestRawCoderBase.java | 13 +-- 17 files changed, 268 insertions(+), 129 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt index 9ccd3a7d3418f..1f3006e14ba9c 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -68,4 +68,7 @@ HADOOP-12065. Using more meaningful keys in EC schema. (Kai Zheng) - HDFS-8557. Allow to configure RS and XOR raw coders (Kai Zheng) \ No newline at end of file + HDFS-8557. Allow to configure RS and XOR raw coders (Kai Zheng) + + HADOOP-12060. Fix ByteBuffer usage for raw erasure coders. (Kai Zheng via + jing9) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java index 5d226248ef214..027d58b0260f6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java @@ -22,17 +22,17 @@ import org.apache.hadoop.io.erasurecode.rawcoder.*; /** - * A codec utility. + * A codec & coder utility to help create raw coders conveniently. */ public final class CodecUtil { - private CodecUtil() {} + private CodecUtil() { } /** * Create RS raw encoder according to configuration. - * @param conf - * @param numDataUnits - * @param numParityUnits + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw encoder */ public static RawErasureEncoder createRSRawEncoder( @@ -49,9 +49,9 @@ public static RawErasureEncoder createRSRawEncoder( /** * Create RS raw decoder according to configuration. - * @param conf - * @param numDataUnits - * @param numParityUnits + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw decoder */ public static RawErasureDecoder createRSRawDecoder( @@ -68,9 +68,9 @@ public static RawErasureDecoder createRSRawDecoder( /** * Create XOR raw encoder according to configuration. - * @param conf - * @param numDataUnits - * @param numParityUnits + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw encoder */ public static RawErasureEncoder createXORRawEncoder( @@ -87,9 +87,9 @@ public static RawErasureEncoder createXORRawEncoder( /** * Create XOR raw decoder according to configuration. - * @param conf - * @param numDataUnits - * @param numParityUnits + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw decoder */ public static RawErasureDecoder createXORRawDecoder( @@ -106,11 +106,11 @@ public static RawErasureDecoder createXORRawDecoder( /** * Create raw coder using specified conf and raw coder factory key. - * @param conf - * @param rawCoderFactoryKey - * @param isEncoder - * @param numDataUnits - * @param numParityUnits + * @param conf configuration possibly with some items to configure the coder + * @param rawCoderFactoryKey configuration key to find the raw coder factory + * @param isEncoder is encoder or not we're going to create + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw coder */ public static RawErasureCoder createRawCoder(Configuration conf, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java index 956954a76720d..5c0a16006f867 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java @@ -37,8 +37,8 @@ public ECBlock() { /** * A constructor specifying isParity and isErased. - * @param isParity - * @param isErased + * @param isParity is a parity block + * @param isErased is erased or not */ public ECBlock(boolean isParity, boolean isErased) { this.isParity = isParity; @@ -47,7 +47,7 @@ public ECBlock(boolean isParity, boolean isErased) { /** * Set true if it's for a parity block. - * @param isParity + * @param isParity is parity or not */ public void setParity(boolean isParity) { this.isParity = isParity; @@ -55,10 +55,10 @@ public void setParity(boolean isParity) { /** * Set true if the block is missing. - * @param isMissing + * @param isErased is erased or not */ - public void setErased(boolean isMissing) { - this.isErased = isMissing; + public void setErased(boolean isErased) { + this.isErased = isErased; } /** @@ -71,7 +71,7 @@ public boolean isParity() { /** * - * @return true if it's missing or corrupt due to erasure, otherwise false + * @return true if it's erased due to erasure, otherwise false */ public boolean isErased() { return isErased; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java index 0a869075ac620..91e4fb8f51731 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java @@ -27,8 +27,8 @@ public class ECBlockGroup { /** * A constructor specifying data blocks and parity blocks. - * @param dataBlocks - * @param parityBlocks + * @param dataBlocks data blocks in the group + * @param parityBlocks parity blocks in the group */ public ECBlockGroup(ECBlock[] dataBlocks, ECBlock[] parityBlocks) { this.dataBlocks = dataBlocks; @@ -81,7 +81,7 @@ public boolean anyErasedParityBlock() { /** * Get erased blocks count - * @return + * @return erased count of blocks */ public int getErasedCount() { int erasedCount = 0; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java index 310c7382659b3..d0120d849c457 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -28,7 +28,7 @@ public class ECChunk { /** * Wrapping a ByteBuffer - * @param buffer + * @param buffer buffer to be wrapped by the chunk */ public ECChunk(ByteBuffer buffer) { this.chunkBuffer = buffer; @@ -36,7 +36,7 @@ public ECChunk(ByteBuffer buffer) { /** * Wrapping a bytes array - * @param buffer + * @param buffer buffer to be wrapped by the chunk */ public ECChunk(byte[] buffer) { this.chunkBuffer = ByteBuffer.wrap(buffer); @@ -52,7 +52,7 @@ public ByteBuffer getBuffer() { /** * Convert an array of this chunks to an array of ByteBuffers - * @param chunks + * @param chunks chunks to convert into buffers * @return an array of ByteBuffers */ public static ByteBuffer[] toBuffers(ECChunk[] chunks) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index 1e07d3d04f761..fb0247619d967 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -94,10 +94,10 @@ public ECSchema(String schemaName, Map allOptions) { /** * Constructor with key parameters provided. - * @param schemaName - * @param codecName - * @param numDataUnits - * @param numParityUnits + * @param schemaName schema name + * @param codecName codec name + * @param numDataUnits number of data units used in the schema + * @param numParityUnits number os parity units used in the schema */ public ECSchema(String schemaName, String codecName, int numDataUnits, int numParityUnits) { @@ -107,11 +107,11 @@ public ECSchema(String schemaName, String codecName, /** * Constructor with key parameters provided. Note the extraOptions may contain * additional information for the erasure codec to interpret further. - * @param schemaName - * @param codecName - * @param numDataUnits - * @param numParityUnits - * @param extraOptions + * @param schemaName schema name + * @param codecName codec name + * @param numDataUnits number of data units used in the schema + * @param numParityUnits number os parity units used in the schema + * @param extraOptions extra options to configure the codec */ public ECSchema(String schemaName, String codecName, int numDataUnits, int numParityUnits, Map extraOptions) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java index 9b10c78182762..fce46f8f04334 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java @@ -43,7 +43,8 @@ * A EC schema loading utility that loads predefined EC schemas from XML file */ public class SchemaLoader { - private static final Logger LOG = LoggerFactory.getLogger(SchemaLoader.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger( + SchemaLoader.class.getName()); /** * Load predefined ec schemas from configuration file. This file is diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java index e6a1542b4035c..4b7461e118da2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -91,37 +91,45 @@ protected byte[] resetBuffer(byte[] buffer, int offset, int len) { } /** - * Check and ensure the buffers are of the length specified by dataLen. - * @param buffers - * @param allowNull - * @param dataLen + * Check and ensure the buffers are of the length specified by dataLen, also + * ensure the buffers are direct buffers or not according to isDirectBuffer. + * @param buffers the buffers to check + * @param allowNull whether to allow any element to be null or not + * @param dataLen the length of data available in the buffer to ensure with + * @param isDirectBuffer is direct buffer or not to ensure with */ - protected void ensureLength(ByteBuffer[] buffers, - boolean allowNull, int dataLen) { - for (int i = 0; i < buffers.length; ++i) { - if (buffers[i] == null && !allowNull) { + protected void ensureLengthAndType(ByteBuffer[] buffers, boolean allowNull, + int dataLen, boolean isDirectBuffer) { + for (ByteBuffer buffer : buffers) { + if (buffer == null && !allowNull) { throw new HadoopIllegalArgumentException( "Invalid buffer found, not allowing null"); - } else if (buffers[i] != null && buffers[i].remaining() != dataLen) { - throw new HadoopIllegalArgumentException( - "Invalid buffer, not of length " + dataLen); + } else if (buffer != null) { + if (buffer.remaining() != dataLen) { + throw new HadoopIllegalArgumentException( + "Invalid buffer, not of length " + dataLen); + } + if (buffer.isDirect() != isDirectBuffer) { + throw new HadoopIllegalArgumentException( + "Invalid buffer, isDirect should be " + isDirectBuffer); + } } } } /** * Check and ensure the buffers are of the length specified by dataLen. - * @param buffers - * @param allowNull - * @param dataLen + * @param buffers the buffers to check + * @param allowNull whether to allow any element to be null or not + * @param dataLen the length of data available in the buffer to ensure with */ protected void ensureLength(byte[][] buffers, boolean allowNull, int dataLen) { - for (int i = 0; i < buffers.length; ++i) { - if (buffers[i] == null && !allowNull) { + for (byte[] buffer : buffers) { + if (buffer == null && !allowNull) { throw new HadoopIllegalArgumentException( "Invalid buffer found, not allowing null"); - } else if (buffers[i] != null && buffers[i].length != dataLen) { + } else if (buffer != null && buffer.length != dataLen) { throw new HadoopIllegalArgumentException( "Invalid buffer not of length " + dataLen); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java index c6105b0921cd7..931cda1ce7409 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -41,14 +41,14 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, checkParameters(inputs, erasedIndexes, outputs); ByteBuffer validInput = findFirstValidInput(inputs); + boolean usingDirectBuffer = validInput.isDirect(); int dataLen = validInput.remaining(); if (dataLen == 0) { return; } - ensureLength(inputs, true, dataLen); - ensureLength(outputs, false, dataLen); + ensureLengthAndType(inputs, true, dataLen, usingDirectBuffer); + ensureLengthAndType(outputs, false, dataLen, usingDirectBuffer); - boolean usingDirectBuffer = validInput.isDirect(); if (usingDirectBuffer) { doDecode(inputs, erasedIndexes, outputs); return; @@ -63,14 +63,14 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, for (int i = 0; i < inputs.length; ++i) { buffer = inputs[i]; if (buffer != null) { - inputOffsets[i] = buffer.position(); + inputOffsets[i] = buffer.arrayOffset() + buffer.position(); newInputs[i] = buffer.array(); } } for (int i = 0; i < outputs.length; ++i) { buffer = outputs[i]; - outputOffsets[i] = buffer.position(); + outputOffsets[i] = buffer.arrayOffset() + buffer.position(); newOutputs[i] = buffer.array(); } @@ -81,7 +81,7 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, buffer = inputs[i]; if (buffer != null) { // dataLen bytes consumed - buffer.position(inputOffsets[i] + dataLen); + buffer.position(buffer.position() + dataLen); } } } @@ -89,7 +89,7 @@ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, /** * Perform the real decoding using Direct ByteBuffer. * @param inputs Direct ByteBuffers expected - * @param erasedIndexes + * @param erasedIndexes indexes of erased units in the inputs array * @param outputs Direct ByteBuffers expected */ protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, @@ -117,12 +117,12 @@ public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) { /** * Perform the real decoding using bytes array, supporting offsets and * lengths. - * @param inputs - * @param inputOffsets - * @param dataLen - * @param erasedIndexes - * @param outputs - * @param outputOffsets + * @param inputs the input byte arrays to read data from + * @param inputOffsets offsets for the input byte arrays to read data from + * @param dataLen how much data are to be read from + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs the output byte arrays to write resultant data into + * @param outputOffsets offsets from which to write resultant data into */ protected abstract void doDecode(byte[][] inputs, int[] inputOffsets, int dataLen, int[] erasedIndexes, @@ -139,12 +139,12 @@ public void decode(ECChunk[] inputs, int[] erasedIndexes, /** * Check and validate decoding parameters, throw exception accordingly. The * checking assumes it's a MDS code. Other code can override this. - * @param inputs - * @param erasedIndexes - * @param outputs + * @param inputs input buffers to check + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs output buffers to check */ - protected void checkParameters(Object[] inputs, int[] erasedIndexes, - Object[] outputs) { + protected void checkParameters(T[] inputs, int[] erasedIndexes, + T[] outputs) { if (inputs.length != getNumParityUnits() + getNumDataUnits()) { throw new IllegalArgumentException("Invalid inputs length"); } @@ -160,8 +160,8 @@ protected void checkParameters(Object[] inputs, int[] erasedIndexes, } int validInputs = 0; - for (int i = 0; i < inputs.length; ++i) { - if (inputs[i] != null) { + for (T input : inputs) { + if (input != null) { validInputs += 1; } } @@ -177,7 +177,7 @@ protected void checkParameters(Object[] inputs, int[] erasedIndexes, * not to read. * @return indexes into inputs array */ - protected int[] getErasedOrNotToReadIndexes(Object[] inputs) { + protected int[] getErasedOrNotToReadIndexes(T[] inputs) { int[] invalidIndexes = new int[inputs.length]; int idx = 0; for (int i = 0; i < inputs.length; i++) { @@ -191,13 +191,13 @@ protected int[] getErasedOrNotToReadIndexes(Object[] inputs) { /** * Find the valid input from all the inputs. - * @param inputs + * @param inputs input buffers to look for valid input * @return the first valid input */ protected static T findFirstValidInput(T[] inputs) { - for (int i = 0; i < inputs.length; i++) { - if (inputs[i] != null) { - return inputs[i]; + for (T input : inputs) { + if (input != null) { + return input; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java index d1faa8c65bebe..a0b3cfef2a167 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -37,14 +37,15 @@ public AbstractRawErasureEncoder(int numDataUnits, int numParityUnits) { @Override public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { checkParameters(inputs, outputs); + + boolean usingDirectBuffer = inputs[0].isDirect(); int dataLen = inputs[0].remaining(); if (dataLen == 0) { return; } - ensureLength(inputs, false, dataLen); - ensureLength(outputs, false, dataLen); + ensureLengthAndType(inputs, false, dataLen, usingDirectBuffer); + ensureLengthAndType(outputs, false, dataLen, usingDirectBuffer); - boolean usingDirectBuffer = inputs[0].isDirect(); if (usingDirectBuffer) { doEncode(inputs, outputs); return; @@ -58,13 +59,13 @@ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { ByteBuffer buffer; for (int i = 0; i < inputs.length; ++i) { buffer = inputs[i]; - inputOffsets[i] = buffer.position(); + inputOffsets[i] = buffer.arrayOffset() + buffer.position(); newInputs[i] = buffer.array(); } for (int i = 0; i < outputs.length; ++i) { buffer = outputs[i]; - outputOffsets[i] = buffer.position(); + outputOffsets[i] = buffer.arrayOffset() + buffer.position(); newOutputs[i] = buffer.array(); } @@ -102,11 +103,11 @@ public void encode(byte[][] inputs, byte[][] outputs) { /** * Perform the real encoding work using bytes array, supporting offsets * and lengths. - * @param inputs - * @param inputOffsets - * @param dataLen - * @param outputs - * @param outputOffsets + * @param inputs the input byte arrays to read data from + * @param inputOffsets offsets for the input byte arrays to read data from + * @param dataLen how much data are to be read from + * @param outputs the output byte arrays to write resultant data into + * @param outputOffsets offsets from which to write resultant data into */ protected abstract void doEncode(byte[][] inputs, int[] inputOffsets, int dataLen, byte[][] outputs, @@ -121,10 +122,10 @@ public void encode(ECChunk[] inputs, ECChunk[] outputs) { /** * Check and validate decoding parameters, throw exception accordingly. - * @param inputs - * @param outputs + * @param inputs input buffers to check + * @param outputs output buffers to check */ - protected void checkParameters(Object[] inputs, Object[] outputs) { + protected void checkParameters(T[] inputs, T[] outputs) { if (inputs.length != getNumDataUnits()) { throw new HadoopIllegalArgumentException("Invalid inputs length"); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java index 26eddfc6681bc..280daf3119955 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java @@ -26,16 +26,16 @@ public interface RawErasureCoderFactory { /** * Create raw erasure encoder. - * @param numDataUnits - * @param numParityUnits + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw erasure encoder */ public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits); /** * Create raw erasure decoder. - * @param numDataUnits - * @param numParityUnits + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group * @return raw erasure decoder */ public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java index ad7f32d9177e3..e2d01d9c6fa4d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java @@ -33,7 +33,8 @@ public interface RawErasureDecoder extends RawErasureCoder { /** * Decode with inputs and erasedIndexes, generates outputs. * How to prepare for inputs: - * 1. Create an array containing parity units + data units; + * 1. Create an array containing parity units + data units. Please note the + * parity units should be first or before the data units. * 2. Set null in the array locations specified via erasedIndexes to indicate * they're erased and no data are to read from; * 3. Set null in the array locations for extra redundant items, as they're @@ -48,29 +49,39 @@ public interface RawErasureDecoder extends RawErasureCoder { * erasedIndexes = [5] // index of d2 into inputs array * outputs = [a-writable-buffer] * - * @param inputs inputs to read data from + * Note, for both inputs and outputs, no mixing of on-heap buffers and direct + * buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call * @param erasedIndexes indexes of erased units in the inputs array * @param outputs outputs to write into for data generated according to - * erasedIndexes + * erasedIndexes, ready for reading the result data from after + * the call */ public void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs); /** * Decode with inputs and erasedIndexes, generates outputs. More see above. - * @param inputs inputs to read data from + * @param inputs inputs to read data from, contents may change after the call * @param erasedIndexes indexes of erased units in the inputs array * @param outputs outputs to write into for data generated according to - * erasedIndexes + * erasedIndexes, ready for reading the result data from after + * the call */ public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs); /** * Decode with inputs and erasedIndexes, generates outputs. More see above. - * @param inputs inputs to read data from + * + * Note, for both input and output ECChunks, no mixing of on-heap buffers and + * direct buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call * @param erasedIndexes indexes of erased units in the inputs array * @param outputs outputs to write into for data generated according to - * erasedIndexes + * erasedIndexes, ready for reading the result data from after + * the call */ public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java index 974f86ca6ca62..7571f095caa6a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java @@ -31,23 +31,33 @@ public interface RawErasureEncoder extends RawErasureCoder { /** - * Encode with inputs and generates outputs - * @param inputs + * Encode with inputs and generates outputs. + * + * Note, for both inputs and outputs, no mixing of on-heap buffers and direct + * buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call * @param outputs */ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs); /** * Encode with inputs and generates outputs - * @param inputs - * @param outputs + * @param inputs inputs to read data from, contents may change after the call + * @param outputs outputs to write into for data generated, ready for reading + * the result data from after the call */ public void encode(byte[][] inputs, byte[][] outputs); /** - * Encode with inputs and generates outputs - * @param inputs - * @param outputs + * Encode with inputs and generates outputs. + * + * Note, for both input and output ECChunks, no mixing of on-heap buffers and + * direct buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call + * @param outputs outputs to write into for data generated, ready for reading + * the result data from after the call */ public void encode(ECChunk[] inputs, ECChunk[] outputs); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java new file mode 100644 index 0000000000000..8f552b753eae7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java @@ -0,0 +1,91 @@ +/** + * 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.io.erasurecode; + + +import java.nio.ByteBuffer; + +/** + * An abstract buffer allocator used for test. + */ +public abstract class BufferAllocator { + private boolean usingDirect = false; + + public BufferAllocator(boolean usingDirect) { + this.usingDirect = usingDirect; + } + + protected boolean isUsingDirect() { + return usingDirect; + } + + /** + * Allocate and return a ByteBuffer of specified length. + * @param bufferLen + * @return + */ + public abstract ByteBuffer allocate(int bufferLen); + + /** + * A simple buffer allocator that just uses ByteBuffer's + * allocate/allocateDirect API. + */ + public static class SimpleBufferAllocator extends BufferAllocator { + + public SimpleBufferAllocator(boolean usingDirect) { + super(usingDirect); + } + + @Override + public ByteBuffer allocate(int bufferLen) { + return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) : + ByteBuffer.allocate(bufferLen); + } + } + + /** + * A buffer allocator that allocates a buffer from an existing large buffer by + * slice calling, but if no available space just degrades as + * SimpleBufferAllocator. So please ensure enough space for it. + */ + public static class SlicedBufferAllocator extends BufferAllocator { + private ByteBuffer overallBuffer; + + public SlicedBufferAllocator(boolean usingDirect, int totalBufferLen) { + super(usingDirect); + overallBuffer = isUsingDirect() ? + ByteBuffer.allocateDirect(totalBufferLen) : + ByteBuffer.allocate(totalBufferLen); + } + + @Override + public ByteBuffer allocate(int bufferLen) { + if (bufferLen > overallBuffer.capacity() - overallBuffer.position()) { + // If no available space for the requested length, then allocate new + return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) : + ByteBuffer.allocate(bufferLen); + } + + overallBuffer.limit(overallBuffer.position() + bufferLen); + ByteBuffer result = overallBuffer.slice(); + overallBuffer.position(overallBuffer.position() + bufferLen); + return result; + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java index 10edae8d72f33..8f277f49f82e1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -18,6 +18,8 @@ package org.apache.hadoop.io.erasurecode; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.erasurecode.BufferAllocator.SimpleBufferAllocator; +import org.apache.hadoop.io.erasurecode.BufferAllocator.SlicedBufferAllocator; import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil; import java.nio.ByteBuffer; @@ -40,6 +42,7 @@ public abstract class TestCoderBase { protected int numParityUnits; protected int baseChunkSize = 513; private int chunkSize = baseChunkSize; + private BufferAllocator allocator; private byte[] zeroChunkBytes; @@ -70,6 +73,17 @@ protected void setChunkSize(int chunkSize) { this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default } + protected void prepareBufferAllocator(boolean usingSlicedBuffer) { + if (usingSlicedBuffer) { + int roughEstimationSpace = + chunkSize * (numDataUnits + numParityUnits) * 10; + allocator = new SlicedBufferAllocator(usingDirectBuffer, + roughEstimationSpace); + } else { + allocator = new SimpleBufferAllocator(usingDirectBuffer); + } + } + /** * Set true during setup if want to dump test settings and coding data, * useful in debugging. @@ -299,8 +313,7 @@ protected ByteBuffer allocateOutputBuffer(int bufferLen) { */ int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary int allocLen = startOffset + bufferLen + startOffset; - ByteBuffer buffer = usingDirectBuffer ? - ByteBuffer.allocateDirect(allocLen) : ByteBuffer.allocate(allocLen); + ByteBuffer buffer = allocator.allocate(allocLen); buffer.limit(startOffset + bufferLen); fillDummyData(buffer, startOffset); startBufferWithZero = ! startBufferWithZero; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java index 98fa95614c9d9..738d28e1c4d42 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -65,14 +65,14 @@ protected void testCoding(boolean usingDirectBuffer) { * The following runs will use 3 different chunkSize for inputs and outputs, * to verify the same encoder/decoder can process variable width of data. */ - performTestCoding(baseChunkSize); - performTestCoding(baseChunkSize - 17); - performTestCoding(baseChunkSize + 16); + performTestCoding(baseChunkSize, true); + performTestCoding(baseChunkSize - 17, false); + performTestCoding(baseChunkSize + 16, true); } - private void performTestCoding(int chunkSize) { + private void performTestCoding(int chunkSize, boolean usingSlicedBuffer) { setChunkSize(chunkSize); - + prepareBufferAllocator(usingSlicedBuffer); // Generate data and encode ECBlockGroup blockGroup = prepareBlockGroupForEncoding(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java index 587ce96bf12d0..2b7a3c405b286 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -68,9 +68,9 @@ protected void testCoding(boolean usingDirectBuffer) { * The following runs will use 3 different chunkSize for inputs and outputs, * to verify the same encoder/decoder can process variable width of data. */ - performTestCoding(baseChunkSize, false, false); - performTestCoding(baseChunkSize - 17, false, false); - performTestCoding(baseChunkSize + 16, false, false); + performTestCoding(baseChunkSize, true, false, false); + performTestCoding(baseChunkSize - 17, false, false, false); + performTestCoding(baseChunkSize + 16, true, false, false); } /** @@ -82,7 +82,7 @@ protected void testCodingWithBadInput(boolean usingDirectBuffer) { prepareCoders(); try { - performTestCoding(baseChunkSize, true, false); + performTestCoding(baseChunkSize, false, true, false); Assert.fail("Encoding test with bad input should fail"); } catch (Exception e) { // Expected @@ -98,7 +98,7 @@ protected void testCodingWithBadOutput(boolean usingDirectBuffer) { prepareCoders(); try { - performTestCoding(baseChunkSize, false, true); + performTestCoding(baseChunkSize, false, false, true); Assert.fail("Decoding test with bad output should fail"); } catch (Exception e) { // Expected @@ -122,9 +122,10 @@ public void testCodingWithErasingTooMany() { } } - private void performTestCoding(int chunkSize, + private void performTestCoding(int chunkSize, boolean usingSlicedBuffer, boolean useBadInput, boolean useBadOutput) { setChunkSize(chunkSize); + prepareBufferAllocator(usingSlicedBuffer); dumpSetting(); From f8f7a923b76abcd1d0242c15a536b20af1c1695e Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 21 Jul 2015 15:47:26 -0700 Subject: [PATCH 183/212] HDFS-8760. Erasure Coding: reuse BlockReader when reading the same block in pread. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSInputStream.java | 54 +-- .../hadoop/hdfs/DFSStripedInputStream.java | 350 +++++++----------- .../hadoop/hdfs/server/mover/Mover.java | 3 - .../hadoop/hdfs/util/StripedBlockUtil.java | 35 -- .../hadoop/hdfs/StripedFileTestUtil.java | 17 +- .../hadoop/hdfs/TestWriteReadStripedFile.java | 11 +- 7 files changed, 172 insertions(+), 301 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 4709388662849..10a8cdea433d1 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -364,3 +364,6 @@ to be consistent with trunk. (zhz) HDFS-8433. Erasure coding: set blockToken in LocatedStripedBlock.(waltersu4549) + + HDFS-8760. Erasure Coding: reuse BlockReader when reading the same block in pread. + (jing9) 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 5b10ffe45ad21..6c3f0ee35c040 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 @@ -44,7 +44,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import com.google.common.base.Preconditions; import org.apache.commons.io.IOUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.ByteBufferReadable; @@ -1139,18 +1138,6 @@ public ByteBuffer call() throws Exception { }; } - /** - * Used when reading contiguous blocks - */ - private void actualGetFromOneDataNode(final DNAddrPair datanode, - LocatedBlock block, final long start, final long end, byte[] buf, - int offset, Map> corruptedBlockMap) - throws IOException { - final int length = (int) (end - start + 1); - actualGetFromOneDataNode(datanode, block, start, end, buf, - new int[]{offset}, new int[]{length}, corruptedBlockMap); - } - /** * Read data from one DataNode. * @param datanode the datanode from which to read data @@ -1158,23 +1145,18 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode, * @param startInBlk the startInBlk offset of the block * @param endInBlk the endInBlk offset of the block * @param buf the given byte array into which the data is read - * @param offsets the data may be read into multiple segments of the buf - * (when reading a striped block). this array indicates the - * offset of each buf segment. - * @param lengths the length of each buf segment + * @param offset the offset in buf * @param corruptedBlockMap map recording list of datanodes with corrupted * block replica */ - void actualGetFromOneDataNode(final DNAddrPair datanode, - LocatedBlock block, final long startInBlk, final long endInBlk, - byte[] buf, int[] offsets, int[] lengths, + void actualGetFromOneDataNode(final DNAddrPair datanode, LocatedBlock block, + final long startInBlk, final long endInBlk, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { DFSClientFaultInjector.get().startFetchFromDatanode(); int refetchToken = 1; // only need to get a new access token once int refetchEncryptionKey = 1; // only need to get a new encryption key once final int len = (int) (endInBlk - startInBlk + 1); - checkReadPortions(offsets, lengths, len); while (true) { // cached block locations may have been updated by chooseDataNode() @@ -1186,13 +1168,11 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, DFSClientFaultInjector.get().fetchFromDatanodeException(); reader = getBlockReader(block, startInBlk, len, datanode.addr, datanode.storageType, datanode.info); - for (int i = 0; i < offsets.length; i++) { - int nread = reader.readAll(buf, offsets[i], lengths[i]); - updateReadStatistics(readStatistics, nread, reader); - if (nread != lengths[i]) { - throw new IOException("truncated return from reader.read(): " + - "excpected " + lengths[i] + ", got " + nread); - } + int nread = reader.readAll(buf, offset, len); + updateReadStatistics(readStatistics, nread, reader); + if (nread != len) { + throw new IOException("truncated return from reader.read(): " + + "excpected " + len + ", got " + nread); } DFSClientFaultInjector.get().readFromDatanodeDelay(); return; @@ -1247,24 +1227,6 @@ protected LocatedBlock refreshLocatedBlock(LocatedBlock block) return getBlockAt(block.getStartOffset()); } - /** - * This method verifies that the read portions are valid and do not overlap - * with each other. - */ - private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) { - Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0); - int sum = 0; - for (int i = 0; i < lengths.length; i++) { - if (i > 0) { - int gap = offsets[i] - offsets[i - 1]; - // make sure read portions do not overlap with each other - Preconditions.checkArgument(gap >= lengths[i - 1]); - } - sum += lengths[i]; - } - Preconditions.checkArgument(sum == totalLen); - } - /** * Like {@link #fetchBlockByteRange}except we start up a second, parallel, * 'hedged' read if the first read is taking longer than configured amount of diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 75090036f551a..eecdf67eecd64 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -31,14 +31,6 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.ByteBufferPool; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.finalizeDecodeInputs; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getStartOffsetsForInternalBlocks; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; @@ -48,10 +40,6 @@ import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; -import org.apache.hadoop.net.NetUtils; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; import java.io.EOFException; import java.io.IOException; @@ -166,7 +154,6 @@ void skip() { */ private StripeRange curStripeRange; private final CompletionService readingService; - private ReaderRetryPolicy retry; DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECSchema schema, int cellSize, @@ -198,18 +185,6 @@ private void resetCurStripeBuffer() { curStripeRange = new StripeRange(0, 0); } - @Override - public synchronized int read(final ByteBuffer buf) throws IOException { - ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf); - TraceScope scope = - dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src); - try { - return readWithStrategy(byteBufferReader, 0, buf.remaining()); - } finally { - scope.close(); - } - } - /** * When seeking into a new block group, create blockReader for each internal * block in the group. @@ -229,33 +204,6 @@ private synchronized void blockSeekTo(long target) throws IOException { this.blockEnd = targetBlockGroup.getStartOffset() + targetBlockGroup.getBlockSize() - 1; currentLocatedBlock = targetBlockGroup; - - final long offsetIntoBlockGroup = getOffsetInBlockGroup(); - LocatedBlock[] targetBlocks = parseStripedBlockGroup( - targetBlockGroup, cellSize, dataBlkNum, parityBlkNum); - // The purpose is to get start offset into each block. - long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema, - cellSize, targetBlockGroup, offsetIntoBlockGroup); - Preconditions.checkState(offsetsForInternalBlocks.length == - dataBlkNum + parityBlkNum); - long minOffset = offsetsForInternalBlocks[dataBlkNum]; - - retry = new ReaderRetryPolicy(); - for (int i = 0; i < dataBlkNum; i++) { - LocatedBlock targetBlock = targetBlocks[i]; - if (targetBlock != null) { - DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null); - if (dnInfo != null) { - BlockReader reader = getBlockReaderWithRetry(targetBlock, - minOffset, targetBlock.getBlockSize() - minOffset, - dnInfo.addr, dnInfo.storageType, dnInfo.info, target, retry); - if (reader != null) { - blockReaders[i] = new BlockReaderInfo(reader, targetBlock, - dnInfo.info, minOffset); - } - } - } - } } /** @@ -308,16 +256,16 @@ protected void closeCurrentBlockReaders() { return; } for (int i = 0; i < groupSize; i++) { - closeReader(i); + closeReader(blockReaders[i]); blockReaders[i] = null; } blockEnd = -1; } - private void closeReader(int index) { - if (blockReaders[index] != null) { - IOUtils.cleanup(DFSClient.LOG, blockReaders[index].reader); - blockReaders[index].skip(); + private void closeReader(BlockReaderInfo readerInfo) { + if (readerInfo != null) { + IOUtils.cleanup(DFSClient.LOG, readerInfo.reader); + readerInfo.skip(); } } @@ -358,17 +306,17 @@ private void readOneStripe( for (AlignedStripe stripe : stripes) { // Parse group to get chosen DN location StripeReader sreader = new StatefulStripeReader(readingService, stripe, - blks, corruptedBlockMap); + blks, blockReaders, corruptedBlockMap); sreader.readStripe(); } curStripeBuf.position(stripeBufOffset); curStripeBuf.limit(stripeLimit); } - private Callable readCell(final BlockReader reader, + private Callable readCells(final BlockReader reader, final DatanodeInfo datanode, final long currentReaderOffset, - final long targetReaderOffset, final ByteBufferStrategy strategy, - final int targetLength, final ExtendedBlock currentBlock, + final long targetReaderOffset, final ByteBufferStrategy[] strategies, + final ExtendedBlock currentBlock, final Map> corruptedBlockMap) { return new Callable() { @Override @@ -386,27 +334,31 @@ public Void call() throws Exception { skipped == targetReaderOffset - currentReaderOffset); } int result = 0; - while (result < targetLength) { - int ret = readToBuffer(reader, datanode, strategy, currentBlock, + for (ByteBufferStrategy strategy : strategies) { + result += readToBuffer(reader, datanode, strategy, currentBlock, corruptedBlockMap); - if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); - } - result += ret; } - updateReadStatistics(readStatistics, targetLength, reader); return null; } }; } private int readToBuffer(BlockReader blockReader, - DatanodeInfo currentNode, ByteBufferStrategy readerStrategy, + DatanodeInfo currentNode, ByteBufferStrategy strategy, ExtendedBlock currentBlock, Map> corruptedBlockMap) throws IOException { + final int targetLength = strategy.buf.remaining(); + int length = 0; try { - return readerStrategy.doRead(blockReader, 0, 0); + while (length < targetLength) { + int ret = strategy.doRead(blockReader, 0, 0); + if (ret < 0) { + throw new IOException("Unexpected EOS from the reader"); + } + length += ret; + } + return length; } catch (ChecksumException ce) { DFSClient.LOG.warn("Found Checksum error for " + currentBlock + " from " + currentNode @@ -572,61 +524,49 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, // Refresh the striped block group LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset()); - AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, cellSize, - blockGroup, start, end, buf, offset); + AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes( + schema, cellSize, blockGroup, start, end, buf, offset); CompletionService readService = new ExecutorCompletionService<>( dfsClient.getStripedReadsThreadPool()); final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( blockGroup, cellSize, dataBlkNum, parityBlkNum); - for (AlignedStripe stripe : stripes) { - // Parse group to get chosen DN location - StripeReader preader = new PositionStripeReader(readService, stripe, - blks, corruptedBlockMap); - preader.readStripe(); - } - } - - private Callable getFromOneDataNode(final DNAddrPair datanode, - final LocatedBlock block, final long start, final long end, - final byte[] buf, final int[] offsets, final int[] lengths, - final Map> corruptedBlockMap, - final int hedgedReadId) { - final Span parentSpan = Trace.currentSpan(); - return new Callable() { - @Override - public Void call() throws Exception { - TraceScope scope = - Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan); - try { - actualGetFromOneDataNode(datanode, block, start, - end, buf, offsets, lengths, corruptedBlockMap); - } finally { - scope.close(); - } - return null; + final BlockReaderInfo[] preaderInfos = new BlockReaderInfo[groupSize]; + try { + for (AlignedStripe stripe : stripes) { + // Parse group to get chosen DN location + StripeReader preader = new PositionStripeReader(readService, stripe, + blks, preaderInfos, corruptedBlockMap); + preader.readStripe(); } - }; + } finally { + for (BlockReaderInfo preaderInfo : preaderInfos) { + closeReader(preaderInfo); + } + } } + /** + * The reader for reading a complete {@link AlignedStripe}. Note that an + * {@link AlignedStripe} may cross multiple stripes with cellSize width. + */ private abstract class StripeReader { final Map, Integer> futures = new HashMap<>(); final AlignedStripe alignedStripe; final CompletionService service; final LocatedBlock[] targetBlocks; final Map> corruptedBlockMap; + final BlockReaderInfo[] readerInfos; StripeReader(CompletionService service, AlignedStripe alignedStripe, - LocatedBlock[] targetBlocks, + LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos, Map> corruptedBlockMap) { this.service = service; this.alignedStripe = alignedStripe; this.targetBlocks = targetBlocks; + this.readerInfos = readerInfos; this.corruptedBlockMap = corruptedBlockMap; } - abstract boolean readChunk(final CompletionService service, - final LocatedBlock block, int chunkIndex); - /** prepare all the data chunks */ abstract void prepareDecodeInputs(); @@ -635,7 +575,12 @@ abstract boolean readChunk(final CompletionService service, abstract void decode(); - abstract void updateState4SuccessRead(StripingChunkReadResult result); + void updateState4SuccessRead(StripingChunkReadResult result) { + Preconditions.checkArgument( + result.state == StripingChunkReadResult.SUCCESSFUL); + readerInfos[result.index].setOffset(alignedStripe.getOffsetInBlock() + + alignedStripe.getSpanInBlock()); + } private void checkMissingBlocks() throws IOException { if (alignedStripe.missingChunksNum > parityBlkNum) { @@ -654,7 +599,7 @@ private void readDataForDecoding() throws IOException { for (int i = 0; i < dataBlkNum; i++) { Preconditions.checkNotNull(alignedStripe.chunks[i]); if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) { - if (!readChunk(service, targetBlocks[i], i)) { + if (!readChunk(targetBlocks[i], i)) { alignedStripe.missingChunksNum++; } } @@ -666,7 +611,7 @@ void readParityChunks(int num) throws IOException { for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num; i++) { if (alignedStripe.chunks[i] == null) { - if (prepareParityChunk(i) && readChunk(service, targetBlocks[i], i)) { + if (prepareParityChunk(i) && readChunk(targetBlocks[i], i)) { j++; } else { alignedStripe.missingChunksNum++; @@ -676,12 +621,75 @@ void readParityChunks(int num) throws IOException { checkMissingBlocks(); } + boolean createBlockReader(LocatedBlock block, int chunkIndex) + throws IOException { + DNAddrPair dnInfo = getBestNodeDNAddrPair(block, null); + if (dnInfo != null) { + BlockReader reader = getBlockReaderWithRetry(block, + alignedStripe.getOffsetInBlock(), + block.getBlockSize() - alignedStripe.getOffsetInBlock(), + dnInfo.addr, dnInfo.storageType, dnInfo.info, + block.getStartOffset(), new ReaderRetryPolicy()); + if (reader != null) { + readerInfos[chunkIndex] = new BlockReaderInfo(reader, block, + dnInfo.info, alignedStripe.getOffsetInBlock()); + return true; + } + } + return false; + } + + private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { + if (chunk.byteBuffer != null) { + ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer); + return new ByteBufferStrategy[]{strategy}; + } else { + ByteBufferStrategy[] strategies = + new ByteBufferStrategy[chunk.byteArray.getOffsets().length]; + for (int i = 0; i < strategies.length; i++) { + ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(), + chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]); + strategies[i] = new ByteBufferStrategy(buffer); + } + return strategies; + } + } + + boolean readChunk(final LocatedBlock block, int chunkIndex) + throws IOException { + final StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + if (block == null) { + chunk.state = StripingChunk.MISSING; + return false; + } + if (readerInfos[chunkIndex] == null) { + if (!createBlockReader(block, chunkIndex)) { + chunk.state = StripingChunk.MISSING; + return false; + } + } else if (readerInfos[chunkIndex].shouldSkip) { + chunk.state = StripingChunk.MISSING; + return false; + } + + chunk.state = StripingChunk.PENDING; + Callable readCallable = readCells(readerInfos[chunkIndex].reader, + readerInfos[chunkIndex].datanode, + readerInfos[chunkIndex].blockReaderOffset, + alignedStripe.getOffsetInBlock(), getReadStrategies(chunk), + block.getBlock(), corruptedBlockMap); + + Future request = service.submit(readCallable); + futures.put(request, chunkIndex); + return true; + } + /** read the whole stripe. do decoding if necessary */ void readStripe() throws IOException { for (int i = 0; i < dataBlkNum; i++) { if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { - if (!readChunk(service, targetBlocks[i], i)) { + if (!readChunk(targetBlocks[i], i)) { alignedStripe.missingChunksNum++; } } @@ -700,8 +708,8 @@ void readStripe() throws IOException { // first read failure while (!futures.isEmpty()) { try { - StripingChunkReadResult r = getNextCompletedStripedRead(service, - futures, 0); + StripingChunkReadResult r = StripedBlockUtil + .getNextCompletedStripedRead(service, futures, 0); if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Read task returned: " + r + ", for stripe " + alignedStripe); @@ -721,7 +729,7 @@ void readStripe() throws IOException { } else { returnedChunk.state = StripingChunk.MISSING; // close the corresponding reader - closeReader(r.index); + closeReader(readerInfos[r.index]); final int missing = alignedStripe.missingChunksNum; alignedStripe.missingChunksNum++; @@ -750,48 +758,17 @@ class PositionStripeReader extends StripeReader { PositionStripeReader(CompletionService service, AlignedStripe alignedStripe, LocatedBlock[] targetBlocks, + BlockReaderInfo[] readerInfos, Map> corruptedBlockMap) { - super(service, alignedStripe, targetBlocks, corruptedBlockMap); - } - - @Override - boolean readChunk(final CompletionService service, - final LocatedBlock block, int chunkIndex) { - final StripingChunk chunk = alignedStripe.chunks[chunkIndex]; - if (block == null) { - chunk.state = StripingChunk.MISSING; - return false; - } - DatanodeInfo loc = block.getLocations()[0]; - StorageType type = block.getStorageTypes()[0]; - DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr( - loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())), - type); - chunk.state = StripingChunk.PENDING; - Callable readCallable = getFromOneDataNode(dnAddr, - block, alignedStripe.getOffsetInBlock(), - alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, - chunk.byteArray.buf(), chunk.byteArray.getOffsets(), - chunk.byteArray.getLengths(), corruptedBlockMap, chunkIndex); - Future getFromDNRequest = service.submit(readCallable); - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("Submitting striped read request for " + chunkIndex - + ". Info of the block: " + block + ", offset in block is " - + alignedStripe.getOffsetInBlock() + ", end is " - + (alignedStripe.getOffsetInBlock() - + alignedStripe.getSpanInBlock() - 1)); - } - futures.put(getFromDNRequest, chunkIndex); - return true; + super(service, alignedStripe, targetBlocks, readerInfos, + corruptedBlockMap); } - @Override - void updateState4SuccessRead(StripingChunkReadResult r) {} - @Override void prepareDecodeInputs() { if (decodeInputs == null) { - decodeInputs = initDecodeInputs(alignedStripe, dataBlkNum, parityBlkNum); + decodeInputs = StripedBlockUtil.initDecodeInputs(alignedStripe, + dataBlkNum, parityBlkNum); } } @@ -799,8 +776,8 @@ void prepareDecodeInputs() { boolean prepareParityChunk(int index) { Preconditions.checkState(index >= dataBlkNum && alignedStripe.chunks[index] == null); - final int decodeIndex = convertIndex4Decode(index, dataBlkNum, - parityBlkNum); + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index, + dataBlkNum, parityBlkNum); alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); alignedStripe.chunks[index].addByteArraySlice(0, (int) alignedStripe.getSpanInBlock()); @@ -809,10 +786,10 @@ boolean prepareParityChunk(int index) { @Override void decode() { - finalizeDecodeInputs(decodeInputs, dataBlkNum, parityBlkNum, - alignedStripe); - decodeAndFillBuffer(decodeInputs, alignedStripe, dataBlkNum, - parityBlkNum, decoder); + StripedBlockUtil.finalizeDecodeInputs(decodeInputs, dataBlkNum, + parityBlkNum, alignedStripe); + StripedBlockUtil.decodeAndFillBuffer(decodeInputs, alignedStripe, + dataBlkNum, parityBlkNum, decoder); } } @@ -821,36 +798,10 @@ class StatefulStripeReader extends StripeReader { StatefulStripeReader(CompletionService service, AlignedStripe alignedStripe, LocatedBlock[] targetBlocks, + BlockReaderInfo[] readerInfos, Map> corruptedBlockMap) { - super(service, alignedStripe, targetBlocks, corruptedBlockMap); - } - - @Override - boolean readChunk(final CompletionService service, - final LocatedBlock block, int chunkIndex) { - final StripingChunk chunk = alignedStripe.chunks[chunkIndex]; - final BlockReaderInfo readerInfo = blockReaders[chunkIndex]; - if (readerInfo == null || block == null || readerInfo.shouldSkip) { - chunk.state = StripingChunk.MISSING; - return false; - } - chunk.state = StripingChunk.PENDING; - ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer); - Callable readCallable = readCell(readerInfo.reader, - readerInfo.datanode, readerInfo.blockReaderOffset, - alignedStripe.getOffsetInBlock(), strategy, - chunk.byteBuffer.remaining(), block.getBlock(), corruptedBlockMap); - Future request = readingService.submit(readCallable); - futures.put(request, chunkIndex); - return true; - } - - @Override - void updateState4SuccessRead(StripingChunkReadResult result) { - Preconditions.checkArgument( - result.state == StripingChunkReadResult.SUCCESSFUL); - blockReaders[result.index].setOffset(alignedStripe.getOffsetInBlock() - + alignedStripe.getSpanInBlock()); + super(service, alignedStripe, targetBlocks, readerInfos, + corruptedBlockMap); } @Override @@ -864,8 +815,8 @@ void prepareDecodeInputs() { int pos = (int) (range.offsetInBlock % cellSize + cellSize * i); cur.position(pos); cur.limit((int) (pos + range.spanInBlock)); - final int decodeIndex = convertIndex4Decode(i, dataBlkNum, - parityBlkNum); + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i, + dataBlkNum, parityBlkNum); decodeInputs[decodeIndex] = cur.slice(); if (alignedStripe.chunks[i] == null) { alignedStripe.chunks[i] = new StripingChunk( @@ -884,45 +835,20 @@ boolean prepareParityChunk(int index) throws IOException { // we have failed the block reader before return false; } - final int decodeIndex = convertIndex4Decode(index, dataBlkNum, - parityBlkNum); + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index, + dataBlkNum, parityBlkNum); decodeInputs[decodeIndex] = ByteBuffer.allocateDirect( (int) alignedStripe.range.spanInBlock); alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); - if (blockReaders[index] == null && !prepareParityBlockReader(index)) { - alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING); - return false; - } return true; } - private boolean prepareParityBlockReader(int i) throws IOException { - // prepare the block reader for the parity chunk - LocatedBlock targetBlock = targetBlocks[i]; - if (targetBlock != null) { - final long offsetInBlock = alignedStripe.getOffsetInBlock(); - DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null); - if (dnInfo != null) { - BlockReader reader = getBlockReaderWithRetry(targetBlock, - offsetInBlock, targetBlock.getBlockSize() - offsetInBlock, - dnInfo.addr, dnInfo.storageType, dnInfo.info, - DFSStripedInputStream.this.getPos(), retry); - if (reader != null) { - blockReaders[i] = new BlockReaderInfo(reader, targetBlock, - dnInfo.info, offsetInBlock); - return true; - } - } - } - return false; - } - @Override void decode() { // TODO no copy for data chunks. this depends on HADOOP-12047 final int span = (int) alignedStripe.getSpanInBlock(); for (int i = 0; i < alignedStripe.chunks.length; i++) { - final int decodeIndex = convertIndex4Decode(i, + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i, dataBlkNum, parityBlkNum); if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state == StripingChunk.ALLZERO) { @@ -941,7 +867,7 @@ void decode() { for (int i = 0; i < alignedStripe.chunks.length; i++) { if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state == StripingChunk.MISSING) { - decodeIndices[pos++] = convertIndex4Decode(i, + decodeIndices[pos++] = StripedBlockUtil.convertIndex4Decode(i, dataBlkNum, parityBlkNum); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java index ddfd1ea546c57..dcab0751d8c55 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java @@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.BlockStoragePolicySpi; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.HdfsConfiguration; @@ -53,8 +52,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; - import java.io.BufferedReader; import java.io.FileInputStream; import java.io.IOException; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 9b0939c6dcca4..3e5ef4319860a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -476,41 +476,6 @@ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, return cells; } - /** - * Given a logical start offset in a block group, calculate the physical - * start offset into each stored internal block. - */ - public static long[] getStartOffsetsForInternalBlocks(ECSchema ecSchema, - int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup) { - Preconditions.checkArgument( - rangeStartInBlockGroup < blockGroup.getBlockSize()); - int dataBlkNum = ecSchema.getNumDataUnits(); - int parityBlkNum = ecSchema.getNumParityUnits(); - long[] startOffsets = new long[dataBlkNum + parityBlkNum]; - Arrays.fill(startOffsets, -1L); - int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); - StripingCell firstCell = new StripingCell(ecSchema, cellSize, - firstCellIdxInBG, (int) (rangeStartInBlockGroup % cellSize)); - startOffsets[firstCell.idxInStripe] = - firstCell.idxInInternalBlk * cellSize + firstCell.offset; - long earliestStart = startOffsets[firstCell.idxInStripe]; - for (int i = 1; i < dataBlkNum; i++) { - int idx = firstCellIdxInBG + i; - if (idx * (long) cellSize >= blockGroup.getBlockSize()) { - break; - } - StripingCell cell = new StripingCell(ecSchema, cellSize, idx, 0); - startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * (long) cellSize; - if (startOffsets[cell.idxInStripe] < earliestStart) { - earliestStart = startOffsets[cell.idxInStripe]; - } - } - for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { - startOffsets[i] = earliestStart; - } - return startOffsets; - } - /** * Given a logical byte range, mapped to each {@link StripingCell}, calculate * the physical byte range (inclusive) on each stored internal block. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index 815a50d80a5a3..2866a0ee0c659 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -79,10 +79,19 @@ static void verifyPread(FileSystem fs, Path srcPath, int fileLength, for (int startOffset : startOffsets) { startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); int remaining = fileLength - startOffset; - in.readFully(startOffset, buf, 0, remaining); - for (int i = 0; i < remaining; i++) { - Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + - "same", expected[startOffset + i], buf[i]); + int offset = startOffset; + final byte[] result = new byte[remaining]; + while (remaining > 0) { + int target = Math.min(remaining, buf.length); + in.readFully(offset, buf, 0, target); + System.arraycopy(buf, 0, result, offset - startOffset, target); + remaining -= target; + offset += target; + } + for (int i = 0; i < fileLength - startOffset; i++) { + Assert.assertEquals("Byte at " + (startOffset + i) + " is different, " + + "the startOffset is " + startOffset, + expected[startOffset + i], result[i]); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 2f9322dcdd0c9..089a13438e3e4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -19,13 +19,16 @@ 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.BlockLocation; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; +import org.apache.log4j.Level; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -45,6 +48,11 @@ public class TestWriteReadStripedFile { private static FileSystem fs; private static Configuration conf = new HdfsConfiguration(); + static { + ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class)) + .getLogger().setLevel(Level.ALL); + } + @Before public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); @@ -232,7 +240,8 @@ public void testWriteReadUsingWebHdfs() throws Exception { byte[] smallBuf = new byte[1024]; byte[] largeBuf = new byte[fileLength + 100]; - StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + // TODO: HDFS-8797 + //StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); From 5956d23b645e272748e2edca4c30231e729b96a9 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Tue, 21 Jul 2015 17:00:49 -0700 Subject: [PATCH 184/212] HDFS-8781. Erasure Coding: Correctly handle BlockManager#InvalidateBlocks for striped block. Contributed by Yi Liu. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockManager.java | 33 ++++++-- .../blockmanagement/DatanodeDescriptor.java | 7 ++ .../hdfs/TestReadStripedFileWithDecoding.java | 83 ++++++++++++++----- 4 files changed, 98 insertions(+), 28 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 10a8cdea433d1..ad1e4e77e166f 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -367,3 +367,6 @@ HDFS-8760. Erasure Coding: reuse BlockReader when reading the same block in pread. (jing9) + + HDFS-8781. Erasure Coding: Correctly handle BlockManager#InvalidateBlocks for + striped block. (Yi Liu via jing9) 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 7872baa22e583..1594a9ad41d40 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 @@ -783,7 +783,10 @@ public LocatedBlock convertLastBlockToUnderConstruction( // remove this block from the list of pending blocks to be deleted. for (DatanodeStorageInfo storage : targets) { - invalidateBlocks.remove(storage.getDatanodeDescriptor(), oldBlock); + final Block b = getBlockOnStorage(oldBlock, storage); + if (b != null) { + invalidateBlocks.remove(storage.getDatanodeDescriptor(), b); + } } // Adjust safe-mode totals, since under-construction blocks don't @@ -802,12 +805,14 @@ public LocatedBlock convertLastBlockToUnderConstruction( /** * Get all valid locations of the block */ - private List getValidLocations(Block block) { + private List getValidLocations(BlockInfo block) { final List locations = new ArrayList(blocksMap.numNodes(block)); for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { // filter invalidate replicas - if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) { + Block b = getBlockOnStorage(block, storage); + if(b != null && + !invalidateBlocks.contains(storage.getDatanodeDescriptor(), b)) { locations.add(storage); } } @@ -1156,7 +1161,10 @@ void removeBlocksAssociatedTo(final DatanodeStorageInfo storageInfo) { while(it.hasNext()) { BlockInfo block = it.next(); removeStoredBlock(block, node); - invalidateBlocks.remove(node, block); + final Block b = getBlockOnStorage(block, storageInfo); + if (b != null) { + invalidateBlocks.remove(node, b); + } } namesystem.checkSafeMode(); } @@ -1184,7 +1192,7 @@ private void addToInvalidates(BlockInfo storedBlock) { for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock, State.NORMAL)) { final DatanodeDescriptor node = storage.getDatanodeDescriptor(); - final Block b = getBlockToInvalidate(storedBlock, storage); + final Block b = getBlockOnStorage(storedBlock, storage); if (b != null) { invalidateBlocks.add(b, node, false); datanodes.append(node).append(" "); @@ -1196,7 +1204,7 @@ private void addToInvalidates(BlockInfo storedBlock) { } } - private Block getBlockToInvalidate(BlockInfo storedBlock, + private Block getBlockOnStorage(BlockInfo storedBlock, DatanodeStorageInfo storage) { return storedBlock.isStriped() ? ((BlockInfoStriped) storedBlock).getBlockOnStorage(storage) : storedBlock; @@ -2054,7 +2062,10 @@ private void removeZombieReplicas(BlockReportContext context, // more than one storage on a datanode (and because it's a difficult // assumption to really enforce) removeStoredBlock(block, zombie.getDatanodeDescriptor()); - invalidateBlocks.remove(zombie.getDatanodeDescriptor(), block); + Block b = getBlockOnStorage(block, zombie); + if (b != null) { + invalidateBlocks.remove(zombie.getDatanodeDescriptor(), b); + } } assert(zombie.numBlocks() == 0); LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " + @@ -3273,7 +3284,7 @@ private void processChosenExcessReplica( // should be deleted. Items are removed from the invalidate list // upon giving instructions to the datanodes. // - final Block blockToInvalidate = getBlockToInvalidate(storedBlock, chosen); + final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen); addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor()); blockLog.info("BLOCK* chooseExcessReplicates: " +"({}, {}) is added to invalidated blocks set", chosen, storedBlock); @@ -3838,6 +3849,12 @@ private int invalidateWorkForOneNode(DatanodeInfo dn) { return toInvalidate.size(); } + @VisibleForTesting + public boolean containsInvalidateBlock(final DatanodeInfo dn, + final Block block) { + return invalidateBlocks.contains(dn, block); + } + boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) { if (!this.shouldCheckForEnoughRacks) { return true; 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 47bc7652e0981..e4366c91161f4 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 @@ -694,6 +694,13 @@ public Block[] getInvalidateBlocks(int maxblocks) { } } + @VisibleForTesting + public boolean containsInvalidateBlock(Block block) { + synchronized (invalidateBlocks) { + return invalidateBlocks.contains(block); + } + } + /** * @return Approximate number of blocks currently scheduled to be written */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 34d6034210f7c..8afea198a7ecc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -22,13 +22,16 @@ import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; 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.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; import org.junit.Assert; @@ -274,28 +277,68 @@ public void testReportBadBlock() throws IOException { DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); } - // do stateful read - ByteBuffer result = ByteBuffer.allocate(length); - ByteBuffer buf = ByteBuffer.allocate(1024); - int readLen = 0; - int ret; - try (FSDataInputStream in = fs.open(file)) { - while ((ret = in.read(buf)) >= 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); + try { + // do stateful read + ByteBuffer result = ByteBuffer.allocate(length); + ByteBuffer buf = ByteBuffer.allocate(1024); + int readLen = 0; + int ret; + try (FSDataInputStream in = fs.open(file)) { + while ((ret = in.read(buf)) >= 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + } + Assert.assertEquals("The length of file should be the same to write size", + length, readLen); + Assert.assertArrayEquals(bytes, result.array()); + + // check whether the corruption has been reported to the NameNode + final FSNamesystem ns = cluster.getNamesystem(); + final BlockManager bm = ns.getBlockManager(); + BlockInfo blockInfo = (ns.getFSDirectory().getINode4Write(file.toString()) + .asFile().getBlocks())[0]; + Assert.assertEquals(1, bm.getCorruptReplicas(blockInfo).size()); + } finally { + for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false); } } - Assert.assertEquals("The length of file should be the same to write size", - length, readLen); - Assert.assertArrayEquals(bytes, result.array()); + } + + @Test + public void testInvalidateBlock() throws IOException { + final Path file = new Path("/invalidate"); + final int length = 10; + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, file, bytes); - // check whether the corruption has been reported to the NameNode - final FSNamesystem ns = cluster.getNamesystem(); - final BlockManager bm = ns.getBlockManager(); - BlockInfo blockInfo = (ns.getFSDirectory().getINode4Write(file.toString()) - .asFile().getBlocks())[0]; - Assert.assertEquals(1, bm.getCorruptReplicas(blockInfo).size()); + int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); + Assert.assertNotEquals(-1, dnIndex); + LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() + .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, + cellSize, dataBlocks, parityBlocks); + final Block b = blks[0].getBlock().getLocalBlock(); + + DataNode dn = cluster.getDataNodes().get(dnIndex); + // disable the heartbeat from DN so that the invalidated block record is kept + // in NameNode until heartbeat expires and NN mark the dn as dead + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); + + try { + // delete the file + fs.delete(file, true); + // check the block is added to invalidateBlocks + final FSNamesystem fsn = cluster.getNamesystem(); + final BlockManager bm = fsn.getBlockManager(); + DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId()); + Assert.assertTrue(bm.containsInvalidateBlock( + blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b)); + } finally { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false); + } } } From c2c26e6ea7967f02c284918f68554334d4af8561 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Thu, 23 Jul 2015 11:42:02 -0700 Subject: [PATCH 185/212] HDFS-8813. Erasure Coding: Client no need to decode missing parity blocks. Contributed by Walter Su. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DFSStripedInputStream.java | 7 ++++++- .../java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java | 2 +- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index ad1e4e77e166f..9741585d74da0 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -370,3 +370,6 @@ HDFS-8781. Erasure Coding: Correctly handle BlockManager#InvalidateBlocks for striped block. (Yi Liu via jing9) + + HDFS-8813. Erasure Coding: Client no need to decode missing parity blocks. + (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index eecdf67eecd64..4f3a8eda1da2e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -867,8 +867,13 @@ void decode() { for (int i = 0; i < alignedStripe.chunks.length; i++) { if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state == StripingChunk.MISSING) { - decodeIndices[pos++] = StripedBlockUtil.convertIndex4Decode(i, + int decodeIndex = StripedBlockUtil.convertIndex4Decode(i, dataBlkNum, parityBlkNum); + if (i < dataBlkNum) { + decodeIndices[pos++] = decodeIndex; + } else { + decodeInputs[decodeIndex] = null; + } } } decodeIndices = Arrays.copyOf(decodeIndices, pos); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 3e5ef4319860a..a3ee1e8219c1f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -326,7 +326,7 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, // Step 1: prepare indices and output buffers for missing data units int[] decodeIndices = new int[parityBlkNum]; int pos = 0; - for (int i = 0; i < alignedStripe.chunks.length; i++) { + for (int i = 0; i < dataBlkNum; i++) { if (alignedStripe.chunks[i] != null && alignedStripe.chunks[i].state == StripingChunk.MISSING){ decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum); From 95b499a3671daae9018ae005c9384fb65aa37320 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 24 Jul 2015 13:52:50 -0700 Subject: [PATCH 186/212] HDFS-8798. Erasure Coding: fix DFSStripedInputStream/DFSStripedOutputStream re-fetch token when expired. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/DFSStripedInputStream.java | 93 +++++------ .../hadoop/hdfs/StripedDataStreamer.java | 5 +- .../hadoop/hdfs/util/StripedBlockUtil.java | 4 +- .../hdfs/TestDFSStripedInputStream.java | 1 - ...TestDFSStripedOutputStreamWithFailure.java | 154 +++++++++--------- .../hdfs/server/balancer/TestBalancer.java | 9 + .../TestBlockTokenWithDFSStriped.java | 44 +++-- 8 files changed, 159 insertions(+), 154 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9741585d74da0..2f7a88a2706ee 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -373,3 +373,6 @@ HDFS-8813. Erasure Coding: Client no need to decode missing parity blocks. (Walter Su via jing9) + + HDFS-8798. Erasure Coding: fix DFSStripedInputStream/DFSStripedOutputStream + re-fetch token when expired. (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 4f3a8eda1da2e..1f64d4ed4af17 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -20,7 +20,6 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.ReadOption; -import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -44,7 +43,6 @@ import java.io.EOFException; import java.io.IOException; import java.io.InterruptedIOException; -import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.EnumSet; @@ -206,44 +204,6 @@ private synchronized void blockSeekTo(long target) throws IOException { currentLocatedBlock = targetBlockGroup; } - /** - * @throws IOException only when failing to refetch block token, which happens - * when this client cannot get located block information from NameNode. This - * method returns null instead of throwing exception when failing to connect - * to the DataNode. - */ - private BlockReader getBlockReaderWithRetry(LocatedBlock targetBlock, - long offsetInBlock, long length, InetSocketAddress targetAddr, - StorageType storageType, DatanodeInfo datanode, long offsetInFile, - ReaderRetryPolicy retry) throws IOException { - // only need to get a new access token or a new encryption key once - while (true) { - try { - return getBlockReader(targetBlock, offsetInBlock, length, targetAddr, - storageType, datanode); - } catch (IOException e) { - if (e instanceof InvalidEncryptionKeyException && - retry.shouldRefetchEncryptionKey()) { - DFSClient.LOG.info("Will fetch a new encryption key and retry, " - + "encryption key was invalid when connecting to " + targetAddr - + " : " + e); - dfsClient.clearDataEncryptionKey(); - retry.refetchEncryptionKey(); - } else if (retry.shouldRefetchToken() && - tokenRefetchNeeded(e, targetAddr)) { - fetchBlockAt(offsetInFile); - retry.refetchToken(); - } else { - DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block" - + ", add to deadNodes and continue.", e); - // Put chosen node into dead list, continue - addToDeadNodes(datanode); - return null; - } - } - } - } - /** * Extend the super method with the logic of switching between cells. * When reaching the end of a cell, proceed to the next cell and read it @@ -293,13 +253,13 @@ private void readOneStripe( final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen); final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize() - (stripeIndex * stripeLen), stripeLen); - curStripeRange = new StripeRange(offsetInBlockGroup, + StripeRange stripeRange = new StripeRange(offsetInBlockGroup, stripeLimit - stripeBufOffset); LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock; AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(schema, cellSize, blockGroup, offsetInBlockGroup, - offsetInBlockGroup + curStripeRange.length - 1, curStripeBuf); + offsetInBlockGroup + stripeRange.length - 1, curStripeBuf); final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( blockGroup, cellSize, dataBlkNum, parityBlkNum); // read the whole stripe @@ -311,6 +271,7 @@ private void readOneStripe( } curStripeBuf.position(stripeBufOffset); curStripeBuf.limit(stripeLimit); + curStripeRange = stripeRange; } private Callable readCells(final BlockReader reader, @@ -423,7 +384,6 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, } Map> corruptedBlockMap = new ConcurrentHashMap<>(); - failures = 0; if (pos < getFileLength()) { try { if (pos > blockEnd) { @@ -623,13 +583,46 @@ void readParityChunks(int num) throws IOException { boolean createBlockReader(LocatedBlock block, int chunkIndex) throws IOException { - DNAddrPair dnInfo = getBestNodeDNAddrPair(block, null); - if (dnInfo != null) { - BlockReader reader = getBlockReaderWithRetry(block, - alignedStripe.getOffsetInBlock(), - block.getBlockSize() - alignedStripe.getOffsetInBlock(), - dnInfo.addr, dnInfo.storageType, dnInfo.info, - block.getStartOffset(), new ReaderRetryPolicy()); + BlockReader reader = null; + final ReaderRetryPolicy retry = new ReaderRetryPolicy(); + DNAddrPair dnInfo = new DNAddrPair(null, null, null); + + while(true) { + try { + // the cached block location might have been re-fetched, so always + // get it from cache. + block = refreshLocatedBlock(block); + targetBlocks[chunkIndex] = block; + + // internal block has one location, just rule out the deadNodes + dnInfo = getBestNodeDNAddrPair(block, null); + if (dnInfo == null) { + break; + } + reader = getBlockReader(block, alignedStripe.getOffsetInBlock(), + block.getBlockSize() - alignedStripe.getOffsetInBlock(), + dnInfo.addr, dnInfo.storageType, dnInfo.info); + } catch (IOException e) { + if (e instanceof InvalidEncryptionKeyException && + retry.shouldRefetchEncryptionKey()) { + DFSClient.LOG.info("Will fetch a new encryption key and retry, " + + "encryption key was invalid when connecting to " + dnInfo.addr + + " : " + e); + dfsClient.clearDataEncryptionKey(); + retry.refetchEncryptionKey(); + } else if (retry.shouldRefetchToken() && + tokenRefetchNeeded(e, dnInfo.addr)) { + fetchBlockAt(block.getStartOffset()); + retry.refetchToken(); + } else { + //TODO: handles connection issues + DFSClient.LOG.warn("Failed to connect to " + dnInfo.addr + " for " + + "block" + block.getBlock(), e); + // re-fetch the block in case the block has been moved + fetchBlockAt(block.getStartOffset()); + addToDeadNodes(dnInfo.info); + } + } if (reader != null) { readerInfos[chunkIndex] = new BlockReaderInfo(reader, block, dnInfo.info, alignedStripe.getOffsetInBlock()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index a1777962756a4..2d51dc43588ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -195,12 +195,15 @@ void populate() throws IOException { final ExtendedBlock bg = coordinator.getBlockGroup(); final LocatedBlock updated = callUpdateBlockForPipeline(bg); final long newGS = updated.getBlock().getGenerationStamp(); + final LocatedBlock[] updatedBlks = StripedBlockUtil + .parseStripedBlockGroup((LocatedStripedBlock) updated, + BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock(); if (bi != null) { final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS), null, null, null, -1, updated.isCorrupt(), null); - lb.setBlockToken(updated.getBlockToken()); + lb.setBlockToken(updatedBlks[i].getBlockToken()); newBlocks.offer(i, lb); } else { final LocatedBlock lb = coordinator.getFollowingBlocks().peek(i); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index a3ee1e8219c1f..4dc94a0d7ace4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -119,8 +119,8 @@ public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, bg.getStartOffset(), bg.isCorrupt(), null); } Token[] blockTokens = bg.getBlockTokens(); - if (idxInBlockGroup < blockTokens.length) { - locatedBlock.setBlockToken(blockTokens[idxInBlockGroup]); + if (idxInReturnedLocs < blockTokens.length) { + locatedBlock.setBlockToken(blockTokens[idxInReturnedLocs]); } return locatedBlock; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index c520d2c770d80..baf610654ebc9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.junit.After; import org.junit.Assert; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index 8944cde02de79..54fcdf8c0991b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -25,22 +25,27 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +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.SecurityTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.util.StringUtils; import org.apache.log4j.Level; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import com.google.common.base.Preconditions; @@ -63,17 +68,13 @@ public class TestDFSStripedOutputStreamWithFailure { private static final int FLUSH_POS = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1; - private final HdfsConfiguration conf = new HdfsConfiguration(); private MiniDFSCluster cluster; private DistributedFileSystem dfs; private final Path dir = new Path("/" + TestDFSStripedOutputStreamWithFailure.class.getSimpleName()); - - @Before - public void setup() throws IOException { + private void setup(Configuration conf) throws IOException { final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); @@ -81,8 +82,7 @@ public void setup() throws IOException { dfs.createErasureCodingZone(dir, null, 0); } - @After - public void tearDown() { + private void tearDown() { if (cluster != null) { cluster.shutdown(); } @@ -92,89 +92,76 @@ private static byte getByte(long pos) { return (byte)pos; } - @Test(timeout=120000) - public void testDatanodeFailure0() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 0; - runTest("file" + dn, length, dn); - } - - @Test(timeout=120000) - public void testDatanodeFailure1() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 1; - runTest("file" + dn, length, dn); - } - - @Test(timeout=120000) - public void testDatanodeFailure2() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 2; - runTest("file" + dn, length, dn); - } - - @Test(timeout=120000) - public void testDatanodeFailure3() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 3; - runTest("file" + dn, length, dn); - } - - @Test(timeout=120000) - public void testDatanodeFailure4() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 4; - runTest("file" + dn, length, dn); - } - - @Test(timeout=120000) - public void testDatanodeFailure5() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 5; - runTest("file" + dn, length, dn); - } - - @Test(timeout=120000) - public void testDatanodeFailure6() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 6; - runTest("file" + dn, length, dn); + private void initConf(Configuration conf){ + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); } - @Test(timeout=120000) - public void testDatanodeFailure7() { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 7; - runTest("file" + dn, length, dn); + private void initConfWithBlockToken(Configuration conf) { + conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); + conf.setInt("ipc.client.connect.max.retries", 0); + // Set short retry timeouts so this test runs faster + conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); } - @Test(timeout=120000) - public void testDatanodeFailure8() { + @Test(timeout=240000) + public void testDatanodeFailure() throws Exception { final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - final int dn = 8; - runTest("file" + dn, length, dn); + HdfsConfiguration conf = new HdfsConfiguration(); + initConf(conf); + for (int dn = 0; dn < 9; dn++) { + try { + setup(conf); + cluster.startDataNodes(conf, 1, true, null, null); + cluster.waitActive(); + runTest(new Path(dir, "file" + dn), length, length / 2, dn, false); + } catch (Exception e) { + LOG.error("failed, dn=" + dn + ", length=" + length); + throw e; + } finally { + tearDown(); + } + } } - private void runTest(final String src, final int length, final int dnIndex) { - try { - cluster.startDataNodes(conf, 1, true, null, null); - cluster.waitActive(); - - runTest(new Path(dir, src), length, length/2, dnIndex); - } catch(Exception e) { - LOG.info("FAILED", e); - Assert.fail(StringUtils.stringifyException(e)); + @Test(timeout=240000) + public void testBlockTokenExpired() throws Exception { + final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE); + HdfsConfiguration conf = new HdfsConfiguration(); + initConf(conf); + initConfWithBlockToken(conf); + for (int dn = 0; dn < 9; dn += 2) { + try { + setup(conf); + cluster.startDataNodes(conf, 1, true, null, null); + cluster.waitActive(); + runTest(new Path(dir, "file" + dn), length, length / 2, dn, true); + } catch (Exception e) { + LOG.error("failed, dn=" + dn + ", length=" + length); + throw e; + } finally { + tearDown(); + } } } private void runTest(final Path p, final int length, final int killPos, - final int dnIndex) throws Exception { + final int dnIndex, final boolean tokenExpire) throws Exception { LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos + ", dnIndex=" + dnIndex); Preconditions.checkArgument(killPos < length); Preconditions.checkArgument(killPos > FLUSH_POS); final String fullPath = p.toString(); + final NameNode nn = cluster.getNameNode(); + final BlockManager bm = nn.getNamesystem().getBlockManager(); + final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); + + if (tokenExpire) { + // set a short token lifetime (1 second) + SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); + } + final AtomicInteger pos = new AtomicInteger(); final FSDataOutputStream out = dfs.create(p); final DFSStripedOutputStream stripedOut @@ -189,6 +176,11 @@ private void runTest(final Path p, final int length, final int killPos, Assert.assertTrue(oldGS != -1); Assert.assertEquals(oldGS, gs); + if (tokenExpire) { + DFSTestUtil.flushInternal(stripedOut); + waitTokenExpires(out); + } + killDatanode(cluster, stripedOut, dnIndex, pos); killed = true; } @@ -348,4 +340,14 @@ static void checkData(DistributedFileSystem dfs, String src, int length, killedDnIndex - dataBlockBytes.length); } } + + private void waitTokenExpires(FSDataOutputStream out) throws IOException { + Token token = DFSTestUtil.getBlockToken(out); + while (!SecurityTestUtil.isBlockTokenExpired(token)) { + try { + Thread.sleep(10); + } catch (InterruptedException ignored) { + } + } + } } 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 759eb45771c3f..8239e5f1d6d11 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 @@ -1469,10 +1469,19 @@ public void testManyBalancerSimultaneously() throws Exception { } } + public void integrationTestWithStripedFile(Configuration conf) throws Exception { + initConfWithStripe(conf); + doTestBalancerWithStripedFile(conf); + } + @Test(timeout = 100000) public void testBalancerWithStripedFile() throws Exception { Configuration conf = new Configuration(); initConfWithStripe(conf); + doTestBalancerWithStripedFile(conf); + } + + private void doTestBalancerWithStripedFile(Configuration conf) throws Exception { int numOfDatanodes = dataBlocks + parityBlocks + 2; int numOfRacks = dataBlocks; long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java index e212917b6f7aa..f985f54b57407 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java @@ -20,13 +20,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.balancer.TestBalancer; import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -46,22 +44,6 @@ public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS { FILE_SIZE = BLOCK_SIZE * dataBlocks * 3; } - @Before - public void setup() throws IOException { - conf = getConf(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient() - .createErasureCodingZone("/", null, cellSize); - cluster.waitActive(); - } - - @After - public void tearDown() { - if (cluster != null) { - cluster.shutdown(); - } - } - private Configuration getConf() { Configuration conf = super.getConf(numDNs); conf.setInt("io.bytes.per.checksum", cellSize); @@ -71,14 +53,26 @@ private Configuration getConf() { @Test @Override public void testRead() throws Exception { - //TODO: DFSStripedInputStream handles token expiration -// doTestRead(conf, cluster, true); + conf = getConf(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient() + .createErasureCodingZone("/", null, cellSize); + try { + cluster.waitActive(); + doTestRead(conf, cluster, true); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } } + /** + * tested at {@link org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired()} + */ @Test @Override - public void testWrite() throws Exception { - //TODO: DFSStripedOutputStream handles token expiration + public void testWrite(){ } @Test @@ -90,7 +84,9 @@ public void testAppend() throws Exception { @Test @Override public void testEnd2End() throws Exception { - //TODO: DFSStripedOutputStream handles token expiration + Configuration conf = new Configuration(); + conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); + new TestBalancer().integrationTestWithStripedFile(conf); } @Override From 4a72be6e0e5c14715ebd50a04b711ffc3f160dda Mon Sep 17 00:00:00 2001 From: Walter Su Date: Sat, 25 Jul 2015 10:07:40 +0800 Subject: [PATCH 187/212] HDFS-8769. Erasure coding: unit test for SequentialBlockGroupIdGenerator. Contributed by Rakesh R. --- .../hdfs/protocol/LocatedStripedBlock.java | 2 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../blockmanagement/BlockIdManager.java | 4 + .../TestSequentialBlockGroupId.java | 222 ++++++++++++++++++ 4 files changed, 230 insertions(+), 1 deletion(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java index 6e62220ac9508..a9a80c20c32f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -63,7 +63,7 @@ public String toString() { + "; corrupt=" + isCorrupt() + "; offset=" + getStartOffset() + "; locs=" + Arrays.asList(getLocations()) - + "; indices=" + Arrays.asList(blockIndices) + + "; indices=" + Arrays.toString(blockIndices) + "}"; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2f7a88a2706ee..1ded20316d5c1 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -376,3 +376,6 @@ HDFS-8798. Erasure Coding: fix DFSStripedInputStream/DFSStripedOutputStream re-fetch token when expired. (Walter Su via jing9) + + HDFS-8769. Erasure coding: unit test for SequentialBlockGroupIdGenerator. + (Rakesh R via waltersu4549) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java index 746802c74bba0..685cfcb10d1f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java @@ -248,4 +248,8 @@ public static int getBlockIndex(Block reportedBlock) { return (int) (reportedBlock.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); } + + SequentialBlockGroupIdGenerator getBlockGroupIdGenerator() { + return blockGroupIdGenerator; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java new file mode 100644 index 0000000000000..2f2356f625b68 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java @@ -0,0 +1,222 @@ +/** + * 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 static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +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.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.internal.util.reflection.Whitebox; +import org.mockito.stubbing.Answer; + +/** + * Tests the sequential blockGroup ID generation mechanism and blockGroup ID + * collision handling. + */ +public class TestSequentialBlockGroupId { + private static final Log LOG = LogFactory + .getLog("TestSequentialBlockGroupId"); + + private final short REPLICATION = 1; + private final long SEED = 0; + private final int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; + private final int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + + private final int stripesPerBlock = 2; + private final int blockSize = cellSize * stripesPerBlock; + private final int numDNs = dataBlocks + parityBlocks + 2; + private final int blockGrpCount = 4; + private final int fileLen = blockSize * dataBlocks * blockGrpCount; + + private MiniDFSCluster cluster; + private FileSystem fs; + private SequentialBlockGroupIdGenerator blockGrpIdGenerator; + private Path eczone = new Path("/eczone"); + + @Before + public void setup() throws Exception { + Configuration conf = new HdfsConfiguration(); + conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + + fs = cluster.getFileSystem(); + blockGrpIdGenerator = cluster.getNamesystem().getBlockIdManager() + .getBlockGroupIdGenerator(); + fs.mkdirs(eczone); + cluster.getFileSystem().getClient() + .createErasureCodingZone("/eczone", null, cellSize); + } + + @After + public void teardown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Test that blockGroup IDs are generating unique value. + */ + @Test(timeout = 60000) + public void testBlockGroupIdGeneration() throws IOException { + long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); + + // Create a file that is 4 blocks long. + Path path = new Path(eczone, "testBlockGrpIdGeneration.dat"); + DFSTestUtil.createFile(fs, path, cellSize, fileLen, blockSize, REPLICATION, + SEED); + List blocks = DFSTestUtil.getAllBlocks(fs, path); + assertThat("Wrong BlockGrps", blocks.size(), is(blockGrpCount)); + + // initialising the block group generator for verifying the block id + blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); + // Ensure that the block IDs are generating unique value. + for (int i = 0; i < blocks.size(); ++i) { + blockGrpIdGenerator + .skipTo((blockGrpIdGenerator.getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + + MAX_BLOCKS_IN_GROUP); + long nextBlockExpectedId = blockGrpIdGenerator.getCurrentValue(); + long nextBlockGrpId = blocks.get(i).getBlock().getBlockId(); + LOG.info("BlockGrp" + i + " id is " + nextBlockGrpId); + assertThat("BlockGrpId mismatches!", nextBlockGrpId, + is(nextBlockExpectedId)); + } + } + + /** + * Test that collisions in the blockGroup ID space are handled gracefully. + */ + @Test(timeout = 60000) + public void testTriggerBlockGroupIdCollision() throws IOException { + long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); + + // Create a file with a few blocks to rev up the global block ID + // counter. + Path path1 = new Path(eczone, "testBlockGrpIdCollisionDetection_file1.dat"); + DFSTestUtil.createFile(fs, path1, cellSize, fileLen, blockSize, + REPLICATION, SEED); + List blocks1 = DFSTestUtil.getAllBlocks(fs, path1); + assertThat("Wrong BlockGrps", blocks1.size(), is(blockGrpCount)); + + // Rewind the block ID counter in the name system object. This will result + // in block ID collisions when we try to allocate new blocks. + blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); + + // Trigger collisions by creating a new file. + Path path2 = new Path(eczone, "testBlockGrpIdCollisionDetection_file2.dat"); + DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize, + REPLICATION, SEED); + List blocks2 = DFSTestUtil.getAllBlocks(fs, path2); + assertThat("Wrong BlockGrps", blocks2.size(), is(blockGrpCount)); + + // Make sure that file1 and file2 block IDs are different + for (LocatedBlock locBlock1 : blocks1) { + long blockId1 = locBlock1.getBlock().getBlockId(); + for (LocatedBlock locBlock2 : blocks2) { + long blockId2 = locBlock2.getBlock().getBlockId(); + assertThat("BlockGrpId mismatches!", blockId1, is(not(blockId2))); + } + } + } + + /** + * Test that collisions in the blockGroup ID when the id is occupied by legacy + * block. + */ + @Test(timeout = 60000) + public void testTriggerBlockGroupIdCollisionWithLegacyBlockId() + throws Exception { + long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); + blockGrpIdGenerator + .skipTo((blockGrpIdGenerator.getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + + MAX_BLOCKS_IN_GROUP); + final long curBlockGroupIdValue = blockGrpIdGenerator.getCurrentValue(); + + // Creates contiguous block with negative blockId so that it would trigger + // collision during blockGroup Id generation + FSNamesystem fsn = cluster.getNamesystem(); + // Replace SequentialBlockIdGenerator with a spy + SequentialBlockIdGenerator blockIdGenerator = spy(fsn.getBlockIdManager() + .getBlockIdGenerator()); + Whitebox.setInternalState(fsn.getBlockIdManager(), "blockIdGenerator", + blockIdGenerator); + SequentialBlockIdGenerator spySequentialBlockIdGenerator = new SequentialBlockIdGenerator( + null) { + @Override + public long nextValue() { + return curBlockGroupIdValue; + } + }; + final Answer delegator = new GenericTestUtils.DelegateAnswer( + spySequentialBlockIdGenerator); + doAnswer(delegator).when(blockIdGenerator).nextValue(); + + Path path1 = new Path("/testCollisionWithLegacyBlock_file1.dat"); + DFSTestUtil.createFile(fs, path1, 1024, REPLICATION, SEED); + + List contiguousBlocks = DFSTestUtil.getAllBlocks(fs, path1); + assertThat(contiguousBlocks.size(), is(1)); + Assert.assertEquals("Unexpected BlockId!", curBlockGroupIdValue, + contiguousBlocks.get(0).getBlock().getBlockId()); + + // Reset back to the initial value to trigger collision + blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); + // Trigger collisions by creating a new file. + Path path2 = new Path(eczone, "testCollisionWithLegacyBlock_file2.dat"); + DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize, + REPLICATION, SEED); + List blocks2 = DFSTestUtil.getAllBlocks(fs, path2); + assertThat("Wrong BlockGrps", blocks2.size(), is(blockGrpCount)); + + // Make sure that file1 and file2 block IDs are different + for (LocatedBlock locBlock1 : contiguousBlocks) { + long blockId1 = locBlock1.getBlock().getBlockId(); + for (LocatedBlock locBlock2 : blocks2) { + long blockId2 = locBlock2.getBlock().getBlockId(); + assertThat("BlockGrpId mismatches!", blockId1, is(not(blockId2))); + } + } + } +} From ba90c02853f85f7ee11d6f37a0c6d0e8a2101ec8 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 31 Jul 2015 12:16:15 -0700 Subject: [PATCH 188/212] HDFS-8202. Improve end to end stirpping file test to add erasure recovering test. Contributed by Xinwei Qin. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/StripedFileTestUtil.java | 65 ++++ ...TestDFSStripedOutputStreamWithFailure.java | 26 +- .../hdfs/TestReadStripedFileWithDecoding.java | 320 +++++++++--------- .../hdfs/TestWriteStripedFileWithFailure.java | 162 +++++++++ 5 files changed, 398 insertions(+), 178 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 1ded20316d5c1..673fbab6abbb8 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -379,3 +379,6 @@ HDFS-8769. Erasure coding: unit test for SequentialBlockGroupIdGenerator. (Rakesh R via waltersu4549) + + HDFS-8202. Improve end to end stirpping file test to add erasure recovering + test. (Xinwei Qin via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index 2866a0ee0c659..ca4b2aabd3a20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -17,10 +17,13 @@ */ package org.apache.hadoop.hdfs; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.web.ByteRangeInputStream; import org.junit.Assert; @@ -29,8 +32,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; public class StripedFileTestUtil { + public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class); + static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; @@ -193,4 +199,63 @@ static void assertSeekAndRead(FSDataInputStream fsdis, int pos, StripedFileTestUtil.getByte(pos + i), buf[i]); } } + + static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, + final int dnIndex, final AtomicInteger pos) { + final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex); + final DatanodeInfo datanode = getDatanodes(s); + LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos); + cluster.stopDataNode(datanode.getXferAddr()); + } + + static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { + for(;;) { + final DatanodeInfo[] datanodes = streamer.getNodes(); + if (datanodes != null) { + Assert.assertEquals(1, datanodes.length); + Assert.assertNotNull(datanodes[0]); + return datanodes[0]; + } + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + return null; + } + } + } + + /** + * Generate n random and different numbers within + * specified non-negative integer range + * @param min minimum of the range + * @param max maximum of the range + * @param n number to be generated + * @return + */ + public static int[] randomArray(int min, int max, int n){ + if (n > (max - min + 1) || max < min || min < 0 || max < 0) { + return null; + } + int[] result = new int[n]; + for (int i = 0; i < n; i++) { + result[i] = -1; + } + + int count = 0; + while(count < n) { + int num = (int) (Math.random() * (max - min)) + min; + boolean flag = true; + for (int j = 0; j < n; j++) { + if(num == result[j]){ + flag = false; + break; + } + } + if(flag){ + result[count] = num; + count++; + } + } + return result; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index 54fcdf8c0991b..6594ae1b47ea0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -181,7 +181,7 @@ private void runTest(final Path p, final int length, final int killPos, waitTokenExpires(out); } - killDatanode(cluster, stripedOut, dnIndex, pos); + StripedFileTestUtil.killDatanode(cluster, stripedOut, dnIndex, pos); killed = true; } @@ -217,30 +217,6 @@ static long getGenerationStamp(DFSStripedOutputStream out) } - static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { - for(;;) { - final DatanodeInfo[] datanodes = streamer.getNodes(); - if (datanodes != null) { - Assert.assertEquals(1, datanodes.length); - Assert.assertNotNull(datanodes[0]); - return datanodes[0]; - } - try { - Thread.sleep(100); - } catch (InterruptedException ignored) { - return null; - } - } - } - - static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, - final int dnIndex, final AtomicInteger pos) { - final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex); - final DatanodeInfo datanode = getDatanodes(s); - LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos); - cluster.stopDataNode(datanode.getXferAddr()); - } - static void checkData(DistributedFileSystem dfs, String src, int length, int killedDnIndex, long oldGS) throws IOException { List> blockGroupList = new ArrayList<>(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 8afea198a7ecc..1719d3f54a1ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -20,10 +20,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -43,13 +44,21 @@ import java.io.IOException; import java.nio.ByteBuffer; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.*; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks; public class TestReadStripedFileWithDecoding { static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class); private MiniDFSCluster cluster; private DistributedFileSystem fs; + private final int smallFileLength = blockSize * dataBlocks - 123; + private final int largeFileLength = blockSize * dataBlocks + 123; + private final int[] fileLengths = {smallFileLength, largeFileLength}; + private final int[] dnFailureNums = {1, 2, 3}; @Before public void setup() throws IOException { @@ -67,82 +76,64 @@ public void tearDown() throws IOException { } } - @Test - public void testReadWithDNFailure1() throws IOException { - testReadWithDNFailure("/foo", cellSize * (dataBlocks + 2), 0); - } - - @Test - public void testReadWithDNFailure2() throws IOException { - testReadWithDNFailure("/foo", cellSize * (dataBlocks + 2), cellSize * 5); - } - - @Test - public void testReadWithDNFailure3() throws IOException { - testReadWithDNFailure("/foo", cellSize * dataBlocks, 0); + /** + * Shutdown tolerable number of Datanode before reading. + * Verify the decoding works correctly. + */ + @Test(timeout=300000) + public void testReadWithDNFailure() throws IOException { + for (int fileLength : fileLengths) { + for (int dnFailureNum : dnFailureNums) { + try { + // setup a new cluster with no dead datanode + setup(); + testReadWithDNFailure(fileLength, dnFailureNum); + } catch (IOException ioe) { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + LOG.error("Failed to read file with DN failure:" + + " fileType = "+ fileType + + ", dnFailureNum = " + dnFailureNum); + } finally { + // tear down the cluster + tearDown(); + } + } + } } /** - * Delete a data block before reading. Verify the decoding works correctly. + * Corrupt tolerable number of block before reading. + * Verify the decoding works correctly. */ - @Test + @Test(timeout=300000) public void testReadCorruptedData() throws IOException { - // create file - final Path file = new Path("/partially_deleted"); - final int length = cellSize * dataBlocks * 2; - final byte[] bytes = StripedFileTestUtil.generateBytes(length); - DFSTestUtil.writeFile(fs, file, bytes); - - // corrupt the first data block - // find the corresponding data node - int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); - Assert.assertNotEquals(-1, dnIndex); - // find the target block - LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() - .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); - final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, - cellSize, dataBlocks, parityBlocks); - // find the target block file - File storageDir = cluster.getInstanceStorageDir(dnIndex, 0); - File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock()); - Assert.assertTrue("Block file does not exist", blkFile.exists()); - // delete the block file - LOG.info("Deliberately removing file " + blkFile.getName()); - Assert.assertTrue("Cannot remove file", blkFile.delete()); - verifyRead(file, length, bytes); + for (int fileLength : fileLengths) { + for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) { + for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) { + String src = "/corrupted_" + dataDelNum + "_" + parityDelNum; + testReadWithBlockCorrupted(src, fileLength, + dataDelNum, parityDelNum, false); + } + } + } } /** - * Corrupt the content of the data block before reading. + * Delete tolerable number of block before reading. + * Verify the decoding works correctly. */ - @Test - public void testReadCorruptedData2() throws IOException { - // create file - final Path file = new Path("/partially_corrupted"); - final int length = cellSize * dataBlocks * 2; - final byte[] bytes = StripedFileTestUtil.generateBytes(length); - DFSTestUtil.writeFile(fs, file, bytes); - - // corrupt the first data block - // find the first data node - int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); - Assert.assertNotEquals(-1, dnIndex); - // find the first data block - LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() - .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); - final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, - cellSize, dataBlocks, parityBlocks); - // find the first block file - File storageDir = cluster.getInstanceStorageDir(dnIndex, 0); - File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock()); - Assert.assertTrue("Block file does not exist", blkFile.exists()); - // corrupt the block file - LOG.info("Deliberately corrupting file " + blkFile.getName()); - try (FileOutputStream out = new FileOutputStream(blkFile)) { - out.write("corruption".getBytes()); + @Test(timeout=300000) + public void testReadCorruptedDataByDeleting() throws IOException { + for (int fileLength : fileLengths) { + for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) { + for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) { + String src = "/deleted_" + dataDelNum + "_" + parityDelNum; + testReadWithBlockCorrupted(src, fileLength, + dataDelNum, parityDelNum, true); + } + } } - - verifyRead(file, length, bytes); } private int findFirstDataNode(Path file, long length) throws IOException { @@ -159,87 +150,45 @@ private int findFirstDataNode(Path file, long length) throws IOException { return -1; } - private void verifyRead(Path file, int length, byte[] expected) + private void verifyRead(Path testPath, int length, byte[] expected) throws IOException { - // pread - try (FSDataInputStream fsdis = fs.open(file)) { - byte[] buf = new byte[length]; - int readLen = fsdis.read(0, buf, 0, buf.length); - Assert.assertEquals("The fileSize of file should be the same to write size", - length, readLen); - Assert.assertArrayEquals(expected, buf); - } - - // stateful read - ByteBuffer result = ByteBuffer.allocate(length); - ByteBuffer buf = ByteBuffer.allocate(1024); - int readLen = 0; - int ret; - try (FSDataInputStream in = fs.open(file)) { - while ((ret = in.read(buf)) >= 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); - } - } - Assert.assertEquals("The length of file should be the same to write size", - length, readLen); - Assert.assertArrayEquals(expected, result.array()); + byte[] buffer = new byte[length + 100]; + StripedFileTestUtil.verifyLength(fs, testPath, length); + StripedFileTestUtil.verifyPread(fs, testPath, length, expected, buffer); + StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, buffer); + StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, + ByteBuffer.allocate(length + 100)); + StripedFileTestUtil.verifySeek(fs, testPath, length); } - private void testReadWithDNFailure(String file, int fileSize, - int startOffsetInFile) throws IOException { - final int failedDNIdx = 2; - Path testPath = new Path(file); - final byte[] bytes = StripedFileTestUtil.generateBytes(fileSize); + private void testReadWithDNFailure(int fileLength, int dnFailureNum) + throws IOException { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + String src = "/dnFailure_" + dnFailureNum + "_" + fileType; + LOG.info("testReadWithDNFailure: file = " + src + + ", fileSize = " + fileLength + + ", dnFailureNum = " + dnFailureNum); + + Path testPath = new Path(src); + final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength); DFSTestUtil.writeFile(fs, testPath, bytes); // shut down the DN that holds an internal data block BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, cellSize); - String name = (locs[0].getNames())[failedDNIdx]; - for (DataNode dn : cluster.getDataNodes()) { - int port = dn.getXferPort(); - if (name.contains(Integer.toString(port))) { - dn.shutdown(); - break; - } - } - - // pread - try (FSDataInputStream fsdis = fs.open(testPath)) { - byte[] buf = new byte[fileSize]; - int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length); - Assert.assertEquals("The fileSize of file should be the same to write size", - fileSize - startOffsetInFile, readLen); - - byte[] expected = new byte[readLen]; - System.arraycopy(bytes, startOffsetInFile, expected, 0, - fileSize - startOffsetInFile); - - for (int i = startOffsetInFile; i < fileSize; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", - expected[i - startOffsetInFile], buf[i - startOffsetInFile]); + for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) { + String name = (locs[0].getNames())[failedDnIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + } } } - // stateful read - ByteBuffer result = ByteBuffer.allocate(fileSize); - ByteBuffer buf = ByteBuffer.allocate(1024); - int readLen = 0; - int ret; - try (FSDataInputStream in = fs.open(testPath)) { - while ((ret = in.read(buf)) >= 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); - } - } - Assert.assertEquals("The length of file should be the same to write size", - fileSize, readLen); - Assert.assertArrayEquals(bytes, result.array()); + // check file length, pread, stateful read and seek + verifyRead(testPath, fileLength, bytes); } /** @@ -279,21 +228,8 @@ public void testReportBadBlock() throws IOException { try { // do stateful read - ByteBuffer result = ByteBuffer.allocate(length); - ByteBuffer buf = ByteBuffer.allocate(1024); - int readLen = 0; - int ret; - try (FSDataInputStream in = fs.open(file)) { - while ((ret = in.read(buf)) >= 0) { - readLen += ret; - buf.flip(); - result.put(buf); - buf.clear(); - } - } - Assert.assertEquals("The length of file should be the same to write size", - length, readLen); - Assert.assertArrayEquals(bytes, result.array()); + StripedFileTestUtil.verifyStatefulRead(fs, file, length, bytes, + ByteBuffer.allocate(1024)); // check whether the corruption has been reported to the NameNode final FSNamesystem ns = cluster.getNamesystem(); @@ -341,4 +277,82 @@ public void testInvalidateBlock() throws IOException { DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false); } } + + /** + * Test reading a file with some blocks(data blocks or parity blocks or both) + * deleted or corrupted. + * @param src file path + * @param fileLength file length + * @param dataBlkDelNum the deleted or corrupted number of data blocks. + * @param parityBlkDelNum the deleted or corrupted number of parity blocks. + * @param deleteBlockFile whether block file is deleted or corrupted. + * true is to delete the block file. + * false is to corrupt the content of the block file. + * @throws IOException + */ + private void testReadWithBlockCorrupted(String src, int fileLength, + int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile) + throws IOException { + LOG.info("testReadWithBlockCorrupted: file = " + src + + ", dataBlkDelNum = " + dataBlkDelNum + + ", parityBlkDelNum = " + parityBlkDelNum + + ", deleteBlockFile? " + deleteBlockFile); + int recoverBlkNum = dataBlkDelNum + parityBlkDelNum; + Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive", + dataBlkDelNum >= 0 && parityBlkDelNum >= 0); + Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " + + "should be between 1 ~ " + parityBlocks, recoverBlkNum <= parityBlocks); + + // write a file with the length of writeLen + Path srcPath = new Path(src); + final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(fs, srcPath, bytes); + + // delete or corrupt some blocks + corruptBlocks(srcPath, dataBlkDelNum, parityBlkDelNum, deleteBlockFile); + + // check the file can be read after some blocks were deleted + verifyRead(srcPath, fileLength, bytes); + } + + private void corruptBlocks(Path srcPath, int dataBlkDelNum, + int parityBlkDelNum, boolean deleteBlockFile) throws IOException { + int recoverBlkNum = dataBlkDelNum + parityBlkDelNum; + + LocatedBlocks locatedBlocks = getLocatedBlocks(srcPath); + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + + int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, dataBlocks, + dataBlkDelNum); + Assert.assertNotNull(delDataBlkIndices); + int[] delParityBlkIndices = StripedFileTestUtil.randomArray(dataBlocks, + dataBlocks + parityBlocks, parityBlkDelNum); + Assert.assertNotNull(delParityBlkIndices); + + int[] delBlkIndices = new int[recoverBlkNum]; + System.arraycopy(delDataBlkIndices, 0, + delBlkIndices, 0, delDataBlkIndices.length); + System.arraycopy(delParityBlkIndices, 0, + delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length); + + ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum]; + for (int i = 0; i < recoverBlkNum; i++) { + delBlocks[i] = StripedBlockUtil + .constructInternalBlock(lastBlock.getBlock(), + cellSize, dataBlocks, delBlkIndices[i]); + if (deleteBlockFile) { + // delete the block file + cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]); + } else { + // corrupt the block file + cluster.corruptBlockOnDataNodes(delBlocks[i]); + } + } + } + + private LocatedBlocks getLocatedBlocks(Path filePath) throws IOException { + return fs.getClient().getLocatedBlocks(filePath.toString(), + 0, Long.MAX_VALUE); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java new file mode 100644 index 0000000000000..54487736004ba --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -0,0 +1,162 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks; + +public class TestWriteStripedFileWithFailure { + public static final Log LOG = LogFactory + .getLog(TestReadStripedFileWithMissingBlocks.class); + private static MiniDFSCluster cluster; + private static FileSystem fs; + private static Configuration conf = new HdfsConfiguration(); + private final int smallFileLength = blockSize * dataBlocks - 123; + private final int largeFileLength = blockSize * dataBlocks + 123; + private final int[] fileLengths = {smallFileLength, largeFileLength}; + + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().createErasureCodingZone("/", + null, cellSize); + fs = cluster.getFileSystem(); + } + + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + // Test writing file with some Datanodes failure + @Test(timeout = 300000) + public void testWriteStripedFileWithDNFailure() throws IOException { + for (int fileLength : fileLengths) { + for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) { + for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) { + try { + // setup a new cluster with no dead datanode + setup(); + writeFileWithDNFailure(fileLength, dataDelNum, parityDelNum); + } catch (IOException ioe) { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + LOG.error("Failed to write file with DN failure:" + + " fileType = "+ fileType + + ", dataDelNum = " + dataDelNum + + ", parityDelNum = " + parityDelNum); + throw ioe; + } finally { + // tear down the cluster + tearDown(); + } + } + } + } + } + + /** + * Test writing a file with shutting down some DNs(data DNs or parity DNs or both). + * @param fileLength file length + * @param dataDNFailureNum the shutdown number of data DNs + * @param parityDNFailureNum the shutdown number of parity DNs + * @throws IOException + */ + private void writeFileWithDNFailure(int fileLength, + int dataDNFailureNum, int parityDNFailureNum) throws IOException { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + String src = "/dnFailure_" + dataDNFailureNum + "_" + parityDNFailureNum + + "_" + fileType; + LOG.info("writeFileWithDNFailure: file = " + src + + ", fileType = " + fileType + + ", dataDNFailureNum = " + dataDNFailureNum + + ", parityDNFailureNum = " + parityDNFailureNum); + + Path srcPath = new Path(src); + final AtomicInteger pos = new AtomicInteger(); + final FSDataOutputStream out = fs.create(srcPath); + final DFSStripedOutputStream stripedOut + = (DFSStripedOutputStream)out.getWrappedStream(); + + int[] dataDNFailureIndices = StripedFileTestUtil.randomArray(0, dataBlocks, + dataDNFailureNum); + Assert.assertNotNull(dataDNFailureIndices); + int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks, + dataBlocks + parityBlocks, dataDNFailureNum); + Assert.assertNotNull(parityDNFailureIndices); + + int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum]; + System.arraycopy(dataDNFailureIndices, 0, failedDataNodes, + 0, dataDNFailureIndices.length); + System.arraycopy(parityDNFailureIndices, 0, failedDataNodes, + dataDNFailureIndices.length, parityDNFailureIndices.length); + + final int killPos = fileLength/2; + for (; pos.get() < fileLength; ) { + final int i = pos.getAndIncrement(); + if (i == killPos) { + for(int failedDn : failedDataNodes) { + StripedFileTestUtil.killDatanode(cluster, stripedOut, failedDn, pos); + } + } + write(out, i); + } + out.close(); + + // make sure the expected number of Datanode have been killed + int dnFailureNum = dataDNFailureNum + parityDNFailureNum; + Assert.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + smallBuf); + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + // delete the file + fs.delete(srcPath, true); + } + + void write(FSDataOutputStream out, int i) throws IOException { + try { + out.write(StripedFileTestUtil.getByte(i)); + } catch (IOException e) { + throw new IOException("Failed at i=" + i, e); + } + } +} \ No newline at end of file From 9312b168e2f152ebfb8a7c7b63e74a819adfc5d2 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 3 Aug 2015 17:03:15 -0700 Subject: [PATCH 189/212] HDFS-8804. Erasure Coding: use DirectBufferPool in DFSStripedInputStream for buffer allocation. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../hadoop/hdfs/DFSStripedInputStream.java | 33 +++++++++++++++++-- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 673fbab6abbb8..f087bb4835161 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -382,3 +382,6 @@ HDFS-8202. Improve end to end stirpping file test to add erasure recovering test. (Xinwei Qin via zhz) + + HDFS-8804. Erasure Coding: use DirectBufferPool in DFSStripedInputStream for + buffer allocation. (jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 1f64d4ed4af17..36120635b211c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -39,6 +39,7 @@ import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.util.DirectBufferPool; import java.io.EOFException; import java.io.IOException; @@ -136,6 +137,8 @@ void skip() { } } + private static final DirectBufferPool bufferPool = new DirectBufferPool(); + private final BlockReaderInfo[] blockReaders; private final int cellSize; private final short dataBlkNum; @@ -143,6 +146,7 @@ void skip() { private final int groupSize; /** the buffer for a complete stripe */ private ByteBuffer curStripeBuf; + private ByteBuffer parityBuf; private final ECSchema schema; private final RawErasureDecoder decoder; @@ -177,12 +181,20 @@ void skip() { private void resetCurStripeBuffer() { if (curStripeBuf == null) { - curStripeBuf = ByteBuffer.allocateDirect(cellSize * dataBlkNum); + curStripeBuf = bufferPool.getBuffer(cellSize * dataBlkNum); } curStripeBuf.clear(); curStripeRange = new StripeRange(0, 0); } + private ByteBuffer getParityBuffer() { + if (parityBuf == null) { + parityBuf = bufferPool.getBuffer(cellSize * parityBlkNum); + } + parityBuf.clear(); + return parityBuf; + } + /** * When seeking into a new block group, create blockReader for each internal * block in the group. @@ -204,6 +216,19 @@ private synchronized void blockSeekTo(long target) throws IOException { currentLocatedBlock = targetBlockGroup; } + @Override + public synchronized void close() throws IOException { + super.close(); + if (curStripeBuf != null) { + bufferPool.returnBuffer(curStripeBuf); + curStripeBuf = null; + } + if (parityBuf != null) { + bufferPool.returnBuffer(parityBuf); + parityBuf = null; + } + } + /** * Extend the super method with the logic of switching between cells. * When reaching the end of a cell, proceed to the next cell and read it @@ -830,8 +855,10 @@ boolean prepareParityChunk(int index) throws IOException { } final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index, dataBlkNum, parityBlkNum); - decodeInputs[decodeIndex] = ByteBuffer.allocateDirect( - (int) alignedStripe.range.spanInBlock); + ByteBuffer buf = getParityBuffer().duplicate(); + buf.position(cellSize * decodeIndex); + buf.limit(cellSize * decodeIndex + (int) alignedStripe.range.spanInBlock); + decodeInputs[decodeIndex] = buf.slice(); alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); return true; } From 4de48211c6f8424e1a603f395a214ecc92f5cc74 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 4 Aug 2015 15:23:39 -0700 Subject: [PATCH 190/212] HDFS-8399. Erasure Coding: unit test the behaviour of BlockManager recovery work for the deleted blocks. Contributed by Rakesh R. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/namenode/TestStripedINodeFile.java | 82 +++++++++++++++++++ 2 files changed, 85 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index f087bb4835161..8e54c762edaac 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -385,3 +385,6 @@ HDFS-8804. Erasure Coding: use DirectBufferPool in DFSStripedInputStream for buffer allocation. (jing9) + + HDFS-8399. Erasure Coding: unit test the behaviour of BlockManager recovery + work for the deleted blocks. (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 9cff6144f6e71..a9b2048416837 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -19,16 +19,24 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; @@ -199,4 +207,78 @@ public void testBlockUCStripedComputeQuotaUsage() // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216. assertEquals(9216, counts.getStorageSpace()); } + + /** + * Test the behavior of striped and contiguous block deletions. + */ + @Test(timeout = 60000) + public void testDeleteOp() throws Exception { + MiniDFSCluster cluster = null; + try { + final int len = 1024; + final Path parentDir = new Path("/parentDir"); + final Path zone = new Path(parentDir, "zone"); + final Path zoneFile = new Path(zone, "zoneFile"); + final Path contiguousFile = new Path(parentDir, "someFile"); + final DistributedFileSystem dfs; + final Configuration conf = new Configuration(); + final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + + HdfsConstants.NUM_PARITY_BLOCKS; + conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2); + + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE) + .build(); + cluster.waitActive(); + + FSNamesystem fsn = cluster.getNamesystem(); + dfs = cluster.getFileSystem(); + dfs.mkdirs(zone); + + // create erasure zone + dfs.createErasureCodingZone(zone, null, 0); + DFSTestUtil.createFile(dfs, zoneFile, len, (short) 1, 0xFEED); + DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED); + final FSDirectory fsd = fsn.getFSDirectory(); + + // Case-1: Verify the behavior of striped blocks + // Get blocks of striped file + INode inodeStriped = fsd.getINode("/parentDir/zone/zoneFile"); + assertTrue("Failed to get INodeFile for /parentDir/zone/zoneFile", + inodeStriped instanceof INodeFile); + INodeFile inodeStripedFile = (INodeFile) inodeStriped; + BlockInfo[] stripedBlks = inodeStripedFile.getBlocks(); + for (BlockInfo blockInfo : stripedBlks) { + assertFalse("Mistakenly marked the block as deleted!", + blockInfo.isDeleted()); + } + + // delete erasure zone directory + dfs.delete(zone, true); + for (BlockInfo blockInfo : stripedBlks) { + assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted()); + } + + // Case-2: Verify the behavior of contiguous blocks + // Get blocks of contiguous file + INode inode = fsd.getINode("/parentDir/someFile"); + assertTrue("Failed to get INodeFile for /parentDir/someFile", + inode instanceof INodeFile); + INodeFile inodeFile = (INodeFile) inode; + BlockInfo[] contiguousBlks = inodeFile.getBlocks(); + for (BlockInfo blockInfo : contiguousBlks) { + assertFalse("Mistakenly marked the block as deleted!", + blockInfo.isDeleted()); + } + + // delete parent directory + dfs.delete(parentDir, true); + for (BlockInfo blockInfo : contiguousBlks) { + assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted()); + } + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } } From 8799363db1c0e0ce0abd4ab68b780092e7dc5263 Mon Sep 17 00:00:00 2001 From: boli2 Date: Wed, 5 Aug 2015 22:11:50 -0400 Subject: [PATCH 191/212] HDFS-8857. Erasure Coding: Fix ArrayIndexOutOfBoundsException in TestWriteStripedFileWithFailure. Contributed by Li Bo --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8e54c762edaac..5ad084bfe2e0b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -388,3 +388,6 @@ HDFS-8399. Erasure Coding: unit test the behaviour of BlockManager recovery work for the deleted blocks. (Rakesh R via zhz) + + HDFS-8857. Erasure Coding: Fix ArrayIndexOutOfBoundsException in + TestWriteStripedFileWithFailure. (Li Bo) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java index 54487736004ba..3679c5fcacdcf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -37,7 +37,7 @@ public class TestWriteStripedFileWithFailure { public static final Log LOG = LogFactory - .getLog(TestReadStripedFileWithMissingBlocks.class); + .getLog(TestWriteStripedFileWithFailure.class); private static MiniDFSCluster cluster; private static FileSystem fs; private static Configuration conf = new HdfsConfiguration(); @@ -114,7 +114,7 @@ private void writeFileWithDNFailure(int fileLength, dataDNFailureNum); Assert.assertNotNull(dataDNFailureIndices); int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks, - dataBlocks + parityBlocks, dataDNFailureNum); + dataBlocks + parityBlocks, parityDNFailureNum); Assert.assertNotNull(parityDNFailureIndices); int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum]; From fbf7e81ca007e009b492e3b99060bbfb74394f46 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Fri, 7 Aug 2015 11:25:20 -0700 Subject: [PATCH 192/212] HDFS-8827. Erasure Coding: Fix NPE when NameNode processes over-replicated striped blocks. Contributed by Walter Su and Takuya Fukudome. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockManager.java | 22 ++- .../TestAddOverReplicatedStripedBlocks.java | 152 +++++++++++++++++- 3 files changed, 168 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 5ad084bfe2e0b..45afd2cea912a 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -391,3 +391,6 @@ HDFS-8857. Erasure Coding: Fix ArrayIndexOutOfBoundsException in TestWriteStripedFileWithFailure. (Li Bo) + + HDFS-8827. Erasure Coding: Fix NPE when NameNode processes over-replicated + striped blocks. (Walter Su and Takuya Fukudome via jing9) 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 1594a9ad41d40..0ceb3921fb90c 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 @@ -3135,14 +3135,13 @@ private void chooseExcessReplicates( assert namesystem.hasWriteLock(); // first form a rack to datanodes map and BlockCollection bc = getBlockCollection(storedBlock); - final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( - bc.getStoragePolicyID()); - final List excessTypes = storagePolicy.chooseExcess( - replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); if (storedBlock.isStriped()) { - chooseExcessReplicasStriped(bc, nonExcess, storedBlock, delNodeHint, - excessTypes); + chooseExcessReplicasStriped(bc, nonExcess, storedBlock, delNodeHint); } else { + final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( + bc.getStoragePolicyID()); + final List excessTypes = storagePolicy.chooseExcess( + replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); chooseExcessReplicasContiguous(bc, nonExcess, storedBlock, replication, addedNode, delNodeHint, excessTypes); } @@ -3216,8 +3215,7 @@ private void chooseExcessReplicasContiguous(BlockCollection bc, private void chooseExcessReplicasStriped(BlockCollection bc, final Collection nonExcess, BlockInfo storedBlock, - DatanodeDescriptor delNodeHint, - List excessTypes) { + DatanodeDescriptor delNodeHint) { assert storedBlock instanceof BlockInfoStriped; BlockInfoStriped sblk = (BlockInfoStriped) storedBlock; short groupSize = sblk.getTotalBlockNum(); @@ -3237,6 +3235,14 @@ private void chooseExcessReplicasStriped(BlockCollection bc, found.set(index); storage2index.put(storage, index); } + // the number of target left replicas equals to the of number of the found + // indices. + int numOfTarget = found.cardinality(); + + final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( + bc.getStoragePolicyID()); + final List excessTypes = storagePolicy.chooseExcess( + (short)numOfTarget, DatanodeStorageInfo.toStorageTypes(nonExcess)); // use delHint only if delHint is duplicated final DatanodeStorageInfo delStorageHint = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java index eaf3435817b7b..337911d0cae11 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java @@ -24,9 +24,14 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; 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.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; +import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.junit.After; import org.junit.Before; @@ -35,6 +40,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.HashSet; +import java.util.List; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -49,7 +55,7 @@ public class TestAddOverReplicatedStripedBlocks { private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; private final short GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - private final int NUM_STRIPE_PER_BLOCK = 1; + private final int NUM_STRIPE_PER_BLOCK = 4; private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; private final int numDNs = GROUP_SIZE + 3; @@ -57,6 +63,8 @@ public class TestAddOverReplicatedStripedBlocks { public void setup() throws IOException { Configuration conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + // disable block recovery + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); SimulatedFSDataset.setFactory(conf); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.waitActive(); @@ -113,4 +121,146 @@ public void testProcessOverReplicatedStripedBlock() throws Exception { filePath.toString(), 0, fileLen); DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); } + + @Test + public void testProcessOverReplicatedSBSmallerThanFullBlocks() + throws Exception { + // Create a EC file which doesn't fill full internal blocks. + int fileLen = CELLSIZE * (DATA_BLK_NUM - 1); + byte[] content = new byte[fileLen]; + DFSTestUtil.writeFile(fs, filePath, new String(content)); + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + long gs = bg.getBlock().getGenerationStamp(); + String bpid = bg.getBlock().getBlockPoolId(); + long groupId = bg.getBlock().getBlockId(); + Block blk = new Block(groupId, BLOCK_SIZE, gs); + cluster.triggerBlockReports(); + List infos = Arrays.asList(bg.getLocations()); + + // let a internal block be over replicated with 2 redundant blocks. + // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks + + // 3 parity blocks + 2 redundant blocks > GROUP_SIZE) + blk.setBlockId(groupId + 2); + List dataNodeList = cluster.getDataNodes(); + for (int i = 0; i < numDNs; i++) { + if (!infos.contains(dataNodeList.get(i).getDatanodeId())) { + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + System.out.println("XXX: inject block into datanode " + i); + } + } + + // update blocksMap + cluster.triggerBlockReports(); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // verify that all internal blocks exists + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); + } + + @Test + public void testProcessOverReplicatedAndCorruptStripedBlock() + throws Exception { + long fileLen = DATA_BLK_NUM * BLOCK_SIZE; + DFSTestUtil.createStripedFile(cluster, filePath, null, 1, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + long gs = bg.getBlock().getGenerationStamp(); + String bpid = bg.getBlock().getBlockPoolId(); + long groupId = bg.getBlock().getBlockId(); + Block blk = new Block(groupId, BLOCK_SIZE, gs); + BlockInfoStriped blockInfo = new BlockInfoStriped(blk, + ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE); + for (int i = 0; i < GROUP_SIZE; i++) { + blk.setBlockId(groupId + i); + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + } + cluster.triggerBlockReports(); + + // let a internal block be corrupt + BlockManager bm = cluster.getNamesystem().getBlockManager(); + List infos = Arrays.asList(bg.getLocations()); + List storages = Arrays.asList(bg.getStorageIDs()); + cluster.getNamesystem().writeLock(); + try { + bm.findAndMarkBlockAsCorrupt(lbs.getLastLocatedBlock().getBlock(), + infos.get(0), storages.get(0), "TEST"); + } finally { + cluster.getNamesystem().writeUnlock(); + } + assertEquals(1, bm.countNodes(blockInfo).corruptReplicas()); + + // let a internal block be over replicated with 2 redundant block. + blk.setBlockId(groupId + 2); + cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid); + cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid); + + // update blocksMap + cluster.triggerBlockReports(); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // verify that all internal blocks exists + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); + } + + @Test + public void testProcessOverReplicatedAndMissingStripedBlock() + throws Exception { + long fileLen = CELLSIZE * DATA_BLK_NUM; + DFSTestUtil.createStripedFile(cluster, filePath, null, 1, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + long gs = bg.getBlock().getGenerationStamp(); + String bpid = bg.getBlock().getBlockPoolId(); + long groupId = bg.getBlock().getBlockId(); + Block blk = new Block(groupId, BLOCK_SIZE, gs); + // only inject GROUP_SIZE - 1 blocks, so there is one block missing + for (int i = 0; i < GROUP_SIZE - 1; i++) { + blk.setBlockId(groupId + i); + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + } + cluster.triggerBlockReports(); + + // let a internal block be over replicated with 2 redundant blocks. + // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks + + // 3 parity blocks + 2 redundant blocks > GROUP_SIZE) + blk.setBlockId(groupId + 2); + cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid); + cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid); + + // update blocksMap + cluster.triggerBlockReports(); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // Since one block is missing, when over-replicated blocks got deleted, + // we are left GROUP_SIZE - 1 blocks. + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); + } + } From 1d37a8812160bb030244a1e6b1c753f962d8d2ed Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 13 Aug 2015 10:04:45 -0700 Subject: [PATCH 193/212] HDFS-8854. Erasure coding: add ECPolicy to replace schema+cellSize in hadoop-hdfs. Contributed by Walter Su. --- .../hadoop/io/erasurecode/ECSchema.java | 42 +---- .../hadoop/io/erasurecode/SchemaLoader.java | 152 ------------------ .../hadoop/io/erasurecode/TestECSchema.java | 6 +- .../io/erasurecode/TestSchemaLoader.java | 74 --------- .../hdfs/client/HdfsClientConfigKeys.java | 4 +- .../hdfs/protocol/ErasureCodingPolicy.java | 93 +++++++++++ .../hadoop/hdfs/protocol/HdfsConstants.java | 4 +- .../hadoop/hdfs/protocol/HdfsFileStatus.java | 17 +- .../hadoop/hdfs/protocol/LocatedBlocks.java | 25 +-- .../SnapshottableDirectoryStatus.java | 2 +- .../hadoop/hdfs/web/JsonUtilClient.java | 4 +- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../org/apache/hadoop/hdfs/DFSClient.java | 20 +-- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 6 - .../apache/hadoop/hdfs/DFSOutputStream.java | 2 +- .../hadoop/hdfs/DFSStripedInputStream.java | 20 +-- .../hadoop/hdfs/DFSStripedOutputStream.java | 10 +- .../hadoop/hdfs/DistributedFileSystem.java | 13 +- .../apache/hadoop/hdfs/client/HdfsAdmin.java | 23 ++- .../hadoop/hdfs/protocol/ClientProtocol.java | 10 +- .../hdfs/protocol/ErasureCodingZone.java | 26 +-- .../hdfs/protocol/HdfsLocatedFileStatus.java | 5 +- ...amenodeProtocolServerSideTranslatorPB.java | 25 ++- .../ClientNamenodeProtocolTranslatorPB.java | 36 ++--- .../hadoop/hdfs/protocolPB/PBHelper.java | 59 ++++--- .../blockmanagement/BlockInfoStriped.java | 37 ++--- .../BlockInfoUnderConstructionStriped.java | 13 +- .../server/blockmanagement/BlockManager.java | 15 +- .../blockmanagement/DatanodeDescriptor.java | 6 +- .../hdfs/server/datanode/StorageLocation.java | 2 +- .../erasurecode/ErasureCodingWorker.java | 11 +- .../hadoop/hdfs/server/mover/Mover.java | 14 +- .../namenode/ErasureCodingPolicyManager.java | 115 +++++++++++++ .../namenode/ErasureCodingSchemaManager.java | 127 --------------- .../namenode/ErasureCodingZoneManager.java | 45 +++--- .../server/namenode/FSDirErasureCodingOp.java | 47 ++---- .../namenode/FSDirStatAndListingOp.java | 18 +-- .../server/namenode/FSDirWriteFileOp.java | 11 +- .../hdfs/server/namenode/FSEditLogLoader.java | 8 +- .../server/namenode/FSImageFormatPBINode.java | 23 +-- .../hdfs/server/namenode/FSNamesystem.java | 52 +++--- .../server/namenode/NameNodeRpcServer.java | 11 +- .../hdfs/server/namenode/NamenodeFsck.java | 10 +- .../protocol/BlockECRecoveryCommand.java | 23 +-- .../hdfs/tools/erasurecode/ECCommand.java | 80 ++++----- .../hadoop/hdfs/util/StripedBlockUtil.java | 56 +++---- .../main/proto/ClientNamenodeProtocol.proto | 4 +- .../src/main/proto/erasurecoding.proto | 17 +- .../hadoop-hdfs/src/main/proto/fsimage.proto | 1 - .../hadoop-hdfs/src/main/proto/hdfs.proto | 21 +-- .../hadoop/cli/TestErasureCodingCLI.java | 1 + .../org/apache/hadoop/hdfs/DFSTestUtil.java | 2 +- .../hadoop/hdfs/TestDFSClientRetries.java | 6 +- .../hdfs/TestDFSStripedInputStream.java | 16 +- .../hdfs/TestDFSStripedOutputStream.java | 2 +- ...TestDFSStripedOutputStreamWithFailure.java | 2 +- .../org/apache/hadoop/hdfs/TestDFSUtil.java | 2 +- .../hadoop/hdfs/TestDatanodeConfig.java | 4 +- .../org/apache/hadoop/hdfs/TestECSchemas.java | 54 ------- .../hadoop/hdfs/TestEncryptionZones.java | 2 +- .../hadoop/hdfs/TestErasureCodingZones.java | 58 +++---- ...a.java => TestFileStatusWithECPolicy.java} | 30 ++-- .../org/apache/hadoop/hdfs/TestLease.java | 4 +- .../hdfs/TestReadStripedFileWithDecoding.java | 3 +- .../TestReadStripedFileWithMissingBlocks.java | 3 +- .../hadoop/hdfs/TestRecoverStripedFile.java | 2 +- .../hdfs/TestSafeModeWithStripedFile.java | 3 +- .../hadoop/hdfs/TestWriteReadStripedFile.java | 3 +- .../hdfs/TestWriteStripedFileWithFailure.java | 3 +- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 34 ++-- .../hdfs/server/balancer/TestBalancer.java | 2 +- .../blockmanagement/TestBlockInfoStriped.java | 14 +- .../TestBlockTokenWithDFSStriped.java | 2 +- .../TestSequentialBlockGroupId.java | 2 +- .../TestUnderReplicatedBlockQueues.java | 19 +-- .../hadoop/hdfs/server/mover/TestMover.java | 2 +- .../TestAddOverReplicatedStripedBlocks.java | 4 +- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 19 +-- .../hdfs/server/namenode/TestFSImage.java | 13 +- .../hadoop/hdfs/server/namenode/TestFsck.java | 6 +- .../namenode/TestQuotaWithStripedBlocks.java | 12 +- .../server/namenode/TestStripedINodeFile.java | 27 ++-- ...stOfflineImageViewerWithStripedBlocks.java | 2 +- .../hdfs/util/TestStripedBlockUtil.java | 12 +- .../apache/hadoop/hdfs/web/TestJsonUtil.java | 2 +- .../test/resources/testErasureCodingConf.xml | 44 ++--- 87 files changed, 747 insertions(+), 1119 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java delete mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java rename hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/{TestFileStatusWithECschema.java => TestFileStatusWithECPolicy.java} (56%) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java index fb0247619d967..0d5bf8f0688a4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -29,12 +29,6 @@ public final class ECSchema { public static final String NUM_PARITY_UNITS_KEY = "numParityUnits"; public static final String CODEC_NAME_KEY = "codec"; - /** - * A friendly and understandable name that can mean what's it, also serves as - * the identifier that distinguish it from other schemas. - */ - private final String schemaName; - /** * The erasure codec name associated. */ @@ -59,14 +53,9 @@ public final class ECSchema { /** * Constructor with schema name and provided all options. Note the options may * contain additional information for the erasure codec to interpret further. - * @param schemaName schema name * @param allOptions all schema options */ - public ECSchema(String schemaName, Map allOptions) { - assert (schemaName != null && ! schemaName.isEmpty()); - - this.schemaName = schemaName; - + public ECSchema(Map allOptions) { if (allOptions == null || allOptions.isEmpty()) { throw new IllegalArgumentException("No schema options are provided"); } @@ -94,33 +83,27 @@ public ECSchema(String schemaName, Map allOptions) { /** * Constructor with key parameters provided. - * @param schemaName schema name * @param codecName codec name * @param numDataUnits number of data units used in the schema * @param numParityUnits number os parity units used in the schema */ - public ECSchema(String schemaName, String codecName, - int numDataUnits, int numParityUnits) { - this(schemaName, codecName, numDataUnits, numParityUnits, null); + public ECSchema(String codecName, int numDataUnits, int numParityUnits) { + this(codecName, numDataUnits, numParityUnits, null); } /** * Constructor with key parameters provided. Note the extraOptions may contain * additional information for the erasure codec to interpret further. - * @param schemaName schema name * @param codecName codec name * @param numDataUnits number of data units used in the schema * @param numParityUnits number os parity units used in the schema * @param extraOptions extra options to configure the codec */ - public ECSchema(String schemaName, String codecName, int numDataUnits, - int numParityUnits, Map extraOptions) { - - assert (schemaName != null && ! schemaName.isEmpty()); + public ECSchema(String codecName, int numDataUnits, int numParityUnits, + Map extraOptions) { assert (codecName != null && ! codecName.isEmpty()); assert (numDataUnits > 0 && numParityUnits > 0); - this.schemaName = schemaName; this.codecName = codecName; this.numDataUnits = numDataUnits; this.numParityUnits = numParityUnits; @@ -153,14 +136,6 @@ private int extractIntOption(String optionKey, Map options) { return result; } - /** - * Get the schema name - * @return schema name - */ - public String getSchemaName() { - return schemaName; - } - /** * Get the codec name * @return codec name @@ -201,7 +176,6 @@ public int getNumParityUnits() { public String toString() { StringBuilder sb = new StringBuilder("ECSchema=["); - sb.append("Name=" + schemaName + ", "); sb.append("Codec=" + codecName + ", "); sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ", "); sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits); @@ -235,9 +209,6 @@ public boolean equals(Object o) { if (numParityUnits != ecSchema.numParityUnits) { return false; } - if (!schemaName.equals(ecSchema.schemaName)) { - return false; - } if (!codecName.equals(ecSchema.codecName)) { return false; } @@ -246,8 +217,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - int result = schemaName.hashCode(); - result = 31 * result + codecName.hashCode(); + int result = codecName.hashCode(); result = 31 * result + extraOptions.hashCode(); result = 31 * result + numDataUnits; result = 31 * result + numParityUnits; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java deleted file mode 100644 index fce46f8f04334..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java +++ /dev/null @@ -1,152 +0,0 @@ -/** - * 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.io.erasurecode; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.Node; -import org.w3c.dom.NodeList; -import org.w3c.dom.Text; -import org.xml.sax.SAXException; - -/** - * A EC schema loading utility that loads predefined EC schemas from XML file - */ -public class SchemaLoader { - private static final Logger LOG = LoggerFactory.getLogger( - SchemaLoader.class.getName()); - - /** - * Load predefined ec schemas from configuration file. This file is - * expected to be in the XML format. - */ - public List loadSchema(String schemaFilePath) { - File confFile = getSchemaFile(schemaFilePath); - if (confFile == null) { - LOG.warn("Not found any predefined EC schema file"); - return Collections.emptyList(); - } - - try { - return loadSchema(confFile); - } catch (ParserConfigurationException e) { - throw new RuntimeException("Failed to load schema file: " + confFile); - } catch (IOException e) { - throw new RuntimeException("Failed to load schema file: " + confFile); - } catch (SAXException e) { - throw new RuntimeException("Failed to load schema file: " + confFile); - } - } - - private List loadSchema(File schemaFile) - throws ParserConfigurationException, IOException, SAXException { - - LOG.info("Loading predefined EC schema file {}", schemaFile); - - // Read and parse the schema file. - DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance(); - dbf.setIgnoringComments(true); - DocumentBuilder builder = dbf.newDocumentBuilder(); - Document doc = builder.parse(schemaFile); - Element root = doc.getDocumentElement(); - - if (!"schemas".equals(root.getTagName())) { - throw new RuntimeException("Bad EC schema config file: " + - "top-level element not "); - } - - NodeList elements = root.getChildNodes(); - List schemas = new ArrayList(); - for (int i = 0; i < elements.getLength(); i++) { - Node node = elements.item(i); - if (node instanceof Element) { - Element element = (Element) node; - if ("schema".equals(element.getTagName())) { - ECSchema schema = loadSchema(element); - schemas.add(schema); - } else { - LOG.warn("Bad element in EC schema configuration file: {}", - element.getTagName()); - } - } - } - - return schemas; - } - - /** - * Path to the XML file containing predefined ec schemas. If the path is - * relative, it is searched for in the classpath. - */ - private File getSchemaFile(String schemaFilePath) { - File schemaFile = new File(schemaFilePath); - if (! schemaFile.isAbsolute()) { - URL url = Thread.currentThread().getContextClassLoader() - .getResource(schemaFilePath); - if (url == null) { - LOG.warn("{} not found on the classpath.", schemaFilePath); - schemaFile = null; - } else if (! url.getProtocol().equalsIgnoreCase("file")) { - throw new RuntimeException( - "EC predefined schema file " + url + - " found on the classpath is not on the local filesystem."); - } else { - schemaFile = new File(url.getPath()); - } - } - - return schemaFile; - } - - /** - * Loads a schema from a schema element in the configuration file - */ - private ECSchema loadSchema(Element element) { - String schemaName = element.getAttribute("name"); - Map ecOptions = new HashMap(); - NodeList fields = element.getChildNodes(); - - for (int i = 0; i < fields.getLength(); i++) { - Node fieldNode = fields.item(i); - if (fieldNode instanceof Element) { - Element field = (Element) fieldNode; - String tagName = field.getTagName(); - String value = ((Text) field.getFirstChild()).getData().trim(); - ecOptions.put(tagName, value); - } - } - - ECSchema schema = new ECSchema(schemaName, ecOptions); - return schema; - } -} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java index c362b9619e1a4..1d399017234c9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java @@ -26,7 +26,6 @@ public class TestECSchema { @Test public void testGoodSchema() { - String schemaName = "goodSchema"; int numDataUnits = 6; int numParityUnits = 3; String codec = "rs"; @@ -39,10 +38,9 @@ public void testGoodSchema() { options.put(ECSchema.CODEC_NAME_KEY, codec); options.put(extraOption, extraOptionValue); - ECSchema schema = new ECSchema(schemaName, options); + ECSchema schema = new ECSchema(options); System.out.println(schema.toString()); - - assertEquals(schemaName, schema.getSchemaName()); + assertEquals(numDataUnits, schema.getNumDataUnits()); assertEquals(numParityUnits, schema.getNumParityUnits()); assertEquals(codec, schema.getCodecName()); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java deleted file mode 100644 index 50d20918894e8..0000000000000 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * 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.io.erasurecode; - -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.io.FileWriter; -import java.io.PrintWriter; -import java.util.List; - -import org.junit.Test; - -public class TestSchemaLoader { - - final static String TEST_DIR = new File(System.getProperty( - "test.build.data", "/tmp")).getAbsolutePath(); - - final static String SCHEMA_FILE = new File(TEST_DIR, "test-ecschema") - .getAbsolutePath(); - - @Test - public void testLoadSchema() throws Exception { - PrintWriter out = new PrintWriter(new FileWriter(SCHEMA_FILE)); - out.println(""); - out.println(""); - out.println(" "); - out.println(" 6"); - out.println(" 3"); - out.println(" RS"); - out.println(" "); - out.println(" "); - out.println(" 10"); - out.println(" 4"); - out.println(" RS"); - out.println(" "); - out.println(""); - out.close(); - - SchemaLoader schemaLoader = new SchemaLoader(); - List schemas = schemaLoader.loadSchema(SCHEMA_FILE); - - assertEquals(2, schemas.size()); - - ECSchema schema1 = schemas.get(0); - assertEquals("RSk6m3", schema1.getSchemaName()); - assertEquals(0, schema1.getExtraOptions().size()); - assertEquals(6, schema1.getNumDataUnits()); - assertEquals(3, schema1.getNumParityUnits()); - assertEquals("RS", schema1.getCodecName()); - - ECSchema schema2 = schemas.get(1); - assertEquals("RSk10m4", schema2.getSchemaName()); - assertEquals(0, schema2.getExtraOptions().size()); - assertEquals(10, schema2.getNumDataUnits()); - assertEquals(4, schema2.getNumParityUnits()); - assertEquals("RS", schema2.getCodecName()); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 6006d71faf6a6..2c4d80f2233dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -183,8 +183,8 @@ interface StripedRead { String THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size"; /** - * With default 6+3 schema, each normal read could span 6 DNs. So this - * default value accommodates 3 read streams + * With default RS-6-3-64k erasure coding policy, each normal read could span + * 6 DNs, so this default value accommodates 3 read streams */ int THREADPOOL_SIZE_DEFAULT = 18; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java new file mode 100644 index 0000000000000..e5dfdff5070d3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.io.erasurecode.ECSchema; + +import java.util.Map; + +/** + * A policy about how to write/read/code an erasure coding file. + */ +public final class ErasureCodingPolicy { + + private final String name; + private final ECSchema schema; + private final int cellSize; + + public ErasureCodingPolicy(String name, ECSchema schema, int cellSize){ + this.name = name; + this.schema = schema; + this.cellSize = cellSize; + } + + public String getName() { + return name; + } + + public ECSchema getSchema() { + return schema; + } + + public int getCellSize() { + return cellSize; + } + + public int getNumDataUnits() { + return schema.getNumDataUnits(); + } + + public int getNumParityUnits() { + return schema.getNumParityUnits(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ErasureCodingPolicy that = (ErasureCodingPolicy) o; + + if (that.getName().equals(name) && that.getCellSize() == cellSize + && that.getSchema().equals(schema)) { + return true; + } + return false; + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + schema.hashCode(); + result = 31 * result + cellSize; + return result; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ErasureCodingPolicy=["); + sb.append("Name=" + name + ", "); + sb.append("Schema=[" + schema.toString() + "], "); + sb.append("CellSize=" + cellSize + " "); + sb.append("]"); + return sb.toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java index a527e233de59f..5f7599f154e8a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java @@ -77,8 +77,8 @@ public final class HdfsConstants { /* * These values correspond to the values used by the system default erasure - * coding schema. - * TODO: to be removed once all places use schema. + * coding policy. + * TODO: get these values from ec policy of the associated INodeFile */ public static final byte NUM_DATA_BLOCKS = 6; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java index 8c902b4aae1d2..6e05ce086f732 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtilClient; -import org.apache.hadoop.io.erasurecode.ECSchema; /** Interface that represents the over the wire information for a file. */ @@ -49,8 +48,7 @@ public class HdfsFileStatus { private final FileEncryptionInfo feInfo; - private final ECSchema ecSchema; - private final int stripeCellSize; + private final ErasureCodingPolicy ecPolicy; // Used by dir, not including dot and dotdot. Always zero for a regular file. private final int childrenNum; @@ -77,7 +75,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo, - byte storagePolicy, ECSchema ecSchema, int stripeCellSize) { + byte storagePolicy, ErasureCodingPolicy ecPolicy) { this.length = length; this.isdir = isdir; this.block_replication = (short)block_replication; @@ -97,8 +95,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, this.childrenNum = childrenNum; this.feInfo = feInfo; this.storagePolicy = storagePolicy; - this.ecSchema = ecSchema; - this.stripeCellSize = stripeCellSize; + this.ecPolicy = ecPolicy; } /** @@ -256,12 +253,8 @@ public final FileEncryptionInfo getFileEncryptionInfo() { return feInfo; } - public ECSchema getECSchema() { - return ecSchema; - } - - public int getStripeCellSize() { - return stripeCellSize; + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; } public final int getChildrenNum() { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java index 735e7b2ec28e1..6e01bbe82f168 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java @@ -24,7 +24,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileEncryptionInfo; -import org.apache.hadoop.io.erasurecode.ECSchema; /** * Collection of blocks with their locations and the file length. @@ -38,8 +37,7 @@ public class LocatedBlocks { private final LocatedBlock lastLocatedBlock; private final boolean isLastBlockComplete; private final FileEncryptionInfo fileEncryptionInfo; - private final ECSchema ecSchema; - private final int stripeCellSize; + private final ErasureCodingPolicy ecPolicy; public LocatedBlocks() { fileLength = 0; @@ -48,22 +46,20 @@ public LocatedBlocks() { lastLocatedBlock = null; isLastBlockComplete = false; fileEncryptionInfo = null; - ecSchema = null; - stripeCellSize = 0; + ecPolicy = null; } public LocatedBlocks(long flength, boolean isUnderConstuction, List blks, LocatedBlock lastBlock, boolean isLastBlockCompleted, FileEncryptionInfo feInfo, - ECSchema ecSchema, int stripeCellSize) { + ErasureCodingPolicy ecPolicy) { fileLength = flength; blocks = blks; underConstruction = isUnderConstuction; this.lastLocatedBlock = lastBlock; this.isLastBlockComplete = isLastBlockCompleted; this.fileEncryptionInfo = feInfo; - this.ecSchema = ecSchema; - this.stripeCellSize = stripeCellSize; + this.ecPolicy = ecPolicy; } /** @@ -120,17 +116,10 @@ public FileEncryptionInfo getFileEncryptionInfo() { } /** - * @return The ECSchema for ErasureCoded file, null otherwise. + * @return The ECPolicy for ErasureCoded file, null otherwise. */ - public ECSchema getECSchema() { - return ecSchema; - } - - /** - * @return Stripe Cell size for ErasureCoded file, 0 otherwise. - */ - public int getStripeCellSize() { - return stripeCellSize; + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java index a6c7b10c91546..813ea266cd07f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java @@ -61,7 +61,7 @@ public SnapshottableDirectoryStatus(long modification_time, long access_time, int snapshotNumber, int snapshotQuota, byte[] parentFullPath) { this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time, access_time, permission, owner, group, null, localName, inodeId, - childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0); + childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); this.snapshotNumber = snapshotNumber; this.snapshotQuota = snapshotQuota; this.parentFullPath = parentFullPath; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java index 11e221be5d280..70f0c4b327901 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java @@ -132,7 +132,7 @@ static HdfsFileStatus toFileStatus(final Map json, boolean includesType) { blockSize, mTime, aTime, permission, owner, group, symlink, DFSUtilClient.string2Bytes(localName), fileId, childrenNum, null, - storagePolicy, null, 0); + storagePolicy, null); } /** Convert a Json map to an ExtendedBlock object. */ @@ -479,7 +479,7 @@ static LocatedBlocks toLocatedBlocks( (Map) m.get("lastLocatedBlock")); final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete"); return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks, - lastLocatedBlock, isLastBlockComplete, null, null, 0); + lastLocatedBlock, isLastBlockComplete, null, null); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 45afd2cea912a..b752d5bdeb741 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -394,3 +394,6 @@ HDFS-8827. Erasure Coding: Fix NPE when NameNode processes over-replicated striped blocks. (Walter Su and Takuya Fukudome via jing9) + + HDFS-8854. Erasure coding: add ECPolicy to replace schema+cellSize in + hadoop-hdfs. (Walter Su via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 87b9f906dcacc..e40f8d0a3b6cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -165,7 +165,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.retry.LossyRetryInvocationHandler; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RemoteException; @@ -1194,10 +1194,10 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) try { LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0); if (locatedBlocks != null) { - ECSchema schema = locatedBlocks.getECSchema(); - if (schema != null) { - return new DFSStripedInputStream(this, src, verifyChecksum, schema, - locatedBlocks.getStripeCellSize(), locatedBlocks); + ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy(); + if (ecPolicy != null) { + return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy, + locatedBlocks); } return new DFSInputStream(this, src, verifyChecksum, locatedBlocks); } else { @@ -3011,12 +3011,12 @@ public RemoteIterator listEncryptionZones() return new EncryptionZoneIterator(namenode, traceSampler); } - public void createErasureCodingZone(String src, ECSchema schema, int cellSize) + public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) throws IOException { checkOpen(); TraceScope scope = getPathTraceScope("createErasureCodingZone", src); try { - namenode.createErasureCodingZone(src, schema, cellSize); + namenode.createErasureCodingZone(src, ecPolicy); } catch (RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, SafeModeException.class, @@ -3138,11 +3138,11 @@ public void checkAccess(String src, FsAction mode) throws IOException { } } - public ECSchema[] getECSchemas() throws IOException { + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { checkOpen(); - TraceScope scope = Trace.startSpan("getECSchemas", traceSampler); + TraceScope scope = Trace.startSpan("getErasureCodingPolicies", traceSampler); try { - return namenode.getECSchemas(); + return namenode.getErasureCodingPolicies(); } finally { scope.close(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 77dd1fdbe883f..af23d56668ea7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -680,12 +680,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final boolean DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_DEFAULT = false; - public static final String DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE = - "dfs.client.striped.read.threadpool.size"; - // With default 3+2 schema, each normal read could span 3 DNs. So this - // default value accommodates 6 read streams - public static final int DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE = 18; - // Slow io warning log threshold settings for dfsclient and datanode. public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY = "dfs.datanode.slow.io.warning.threshold.ms"; 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 9e201ad5d921a..373ebdf2074b3 100755 --- 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 @@ -271,7 +271,7 @@ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, } Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!"); final DFSOutputStream out; - if(stat.getECSchema() != null) { + if(stat.getErasureCodingPolicy() != null) { out = new DFSStripedOutputStream(dfsClient, src, stat, flag, progress, checksum, favoredNodes); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 36120635b211c..2ad63b88d7143 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -36,7 +36,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.util.DirectBufferPool; @@ -147,7 +147,7 @@ void skip() { /** the buffer for a complete stripe */ private ByteBuffer curStripeBuf; private ByteBuffer parityBuf; - private final ECSchema schema; + private final ErasureCodingPolicy ecPolicy; private final RawErasureDecoder decoder; /** @@ -158,15 +158,15 @@ void skip() { private final CompletionService readingService; DFSStripedInputStream(DFSClient dfsClient, String src, - boolean verifyChecksum, ECSchema schema, int cellSize, + boolean verifyChecksum, ErasureCodingPolicy ecPolicy, LocatedBlocks locatedBlocks) throws IOException { super(dfsClient, src, verifyChecksum, locatedBlocks); - assert schema != null; - this.schema = schema; - this.cellSize = cellSize; - dataBlkNum = (short) schema.getNumDataUnits(); - parityBlkNum = (short) schema.getNumParityUnits(); + assert ecPolicy != null; + this.ecPolicy = ecPolicy; + this.cellSize = ecPolicy.getCellSize(); + dataBlkNum = (short) ecPolicy.getNumDataUnits(); + parityBlkNum = (short) ecPolicy.getNumParityUnits(); groupSize = dataBlkNum + parityBlkNum; blockReaders = new BlockReaderInfo[groupSize]; curStripeRange = new StripeRange(0, 0); @@ -282,7 +282,7 @@ private void readOneStripe( stripeLimit - stripeBufOffset); LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock; - AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(schema, cellSize, + AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy, cellSize, blockGroup, offsetInBlockGroup, offsetInBlockGroup + stripeRange.length - 1, curStripeBuf); final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( @@ -510,7 +510,7 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset()); AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes( - schema, cellSize, blockGroup, start, end, buf, offset); + ecPolicy, cellSize, blockGroup, start, end, buf, offset); CompletionService readService = new ExecutorCompletionService<>( dfsClient.getStripedReadsThreadPool()); final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index a4bb49de3f62b..b71e59e383f76 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; @@ -276,10 +276,10 @@ ExtendedBlock getBlock() { LOG.debug("Creating DFSStripedOutputStream for " + src); } - final ECSchema schema = stat.getECSchema(); - final int numParityBlocks = schema.getNumParityUnits(); - cellSize = stat.getStripeCellSize(); - numDataBlocks = schema.getNumDataUnits(); + final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy(); + final int numParityBlocks = ecPolicy.getNumParityUnits(); + cellSize = ecPolicy.getCellSize(); + numDataBlocks = ecPolicy.getNumDataUnits(); numAllBlocks = numDataBlocks + numParityBlocks; encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 6d55922058294..95df90b97237c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -90,7 +90,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.Credentials; @@ -2280,18 +2280,17 @@ public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid) * Create the erasurecoding zone * * @param path Directory to create the ec zone - * @param schema ECSchema for the zone. If not specified default will be used. - * @param cellSize Cellsize for the striped erasure coding + * @param ecPolicy erasure coding policy for the zone. If not specified default will be used. * @throws IOException */ - public void createErasureCodingZone(final Path path, final ECSchema schema, - final int cellSize) throws IOException { + public void createErasureCodingZone(final Path path, final ErasureCodingPolicy ecPolicy) + throws IOException { Path absF = fixRelativePart(path); new FileSystemLinkResolver() { @Override public Void doCall(final Path p) throws IOException, UnresolvedLinkException { - dfs.createErasureCodingZone(getPathName(p), schema, cellSize); + dfs.createErasureCodingZone(getPathName(p), ecPolicy); return null; } @@ -2299,7 +2298,7 @@ public Void doCall(final Path p) throws IOException, public Void next(final FileSystem fs, final Path p) throws IOException { if (fs instanceof DistributedFileSystem) { DistributedFileSystem myDfs = (DistributedFileSystem) fs; - myDfs.createErasureCodingZone(p, schema, cellSize); + myDfs.createErasureCodingZone(p, ecPolicy); return null; } throw new UnsupportedOperationException( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java index 5a3c885b49bbd..e6e67cbbe2c17 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.hdfs.tools.DFSAdmin; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; /** * The public API for performing administrative functions on HDFS. Those writing @@ -369,17 +369,13 @@ public void setStoragePolicy(final Path src, final String policyName) /** * Create the ErasureCoding zone * - * @param path - * Directory to create the ErasureCoding zone - * @param schema - * ECSchema for the zone. If not specified default will be used. - * @param cellSize - * Cellsize for the striped ErasureCoding + * @param path Directory to create the ErasureCoding zone + * @param ecPolicy erasure coding policy for the zone. If null, the default will be used. * @throws IOException */ - public void createErasureCodingZone(final Path path, final ECSchema schema, - final int cellSize) throws IOException { - dfs.createErasureCodingZone(path, schema, cellSize); + public void createErasureCodingZone(final Path path, + final ErasureCodingPolicy ecPolicy) throws IOException { + dfs.createErasureCodingZone(path, ecPolicy); } /** @@ -395,12 +391,11 @@ public ErasureCodingZone getErasureCodingZone(final Path path) } /** - * Get the ErasureCoding schemas supported. + * Get the ErasureCoding policies supported. * - * @return ECSchemas * @throws IOException */ - public ECSchema[] getECSchemas() throws IOException { - return dfs.getClient().getECSchemas(); + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { + return dfs.getClient().getErasureCodingPolicies(); } } 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 d2b8a517e752e..cd0453f86047e 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 @@ -54,7 +54,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.retry.AtMostOnce; import org.apache.hadoop.io.retry.Idempotent; import org.apache.hadoop.security.AccessControlException; @@ -1459,21 +1458,20 @@ public List listXAttrs(String src) public EventBatchList getEditsFromTxid(long txid) throws IOException; /** - * Create an erasure coding zone with specified schema, if any, otherwise + * Create an erasure coding zone with specified policy, if any, otherwise * default */ @AtMostOnce - public void createErasureCodingZone(String src, ECSchema schema, int cellSize) + public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) throws IOException; /** - * Gets list of ECSchemas loaded in Namenode + * Get the erasure coding policies loaded in Namenode * - * @return Returns the list of ECSchemas loaded at Namenode * @throws IOException */ @Idempotent - public ECSchema[] getECSchemas() throws IOException; + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException; /** * Get the information about the EC zone for the path diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java index 655def3da53d4..533b630893c15 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java @@ -16,21 +16,17 @@ */ package org.apache.hadoop.hdfs.protocol; -import org.apache.hadoop.io.erasurecode.ECSchema; - /** * Information about the EC Zone at the specified path. */ public class ErasureCodingZone { private String dir; - private ECSchema schema; - private int cellSize; + private ErasureCodingPolicy ecPolicy; - public ErasureCodingZone(String dir, ECSchema schema, int cellSize) { + public ErasureCodingZone(String dir, ErasureCodingPolicy ecPolicy) { this.dir = dir; - this.schema = schema; - this.cellSize = cellSize; + this.ecPolicy = ecPolicy; } /** @@ -43,24 +39,16 @@ public String getDir() { } /** - * Get the schema for the EC Zone + * Get the erasure coding policy for the EC Zone * * @return */ - public ECSchema getSchema() { - return schema; - } - - /** - * Get cellSize for the EC Zone - */ - public int getCellSize() { - return cellSize; + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; } @Override public String toString() { - return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: " - + cellSize; + return "Dir: " + getDir() + ", Policy: " + ecPolicy; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java index 4701538de9e7e..2121dcf23bca9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtilClient; -import org.apache.hadoop.io.erasurecode.ECSchema; /** * Interface that represents the over the wire information @@ -60,10 +59,10 @@ public HdfsLocatedFileStatus(long length, boolean isdir, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy, - ECSchema schema, int stripeCellSize) { + ErasureCodingPolicy ecPolicy) { super(length, isdir, block_replication, blocksize, modification_time, access_time, permission, owner, group, symlink, path, fileId, - childrenNum, feInfo, storagePolicy, schema, stripeCellSize); + childrenNum, feInfo, storagePolicy, ecPolicy); this.locations = locations; } 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 4228a65838a78..d55d00bcef5fd 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 @@ -199,8 +199,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; @@ -220,7 +220,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto; import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; @@ -1403,10 +1403,9 @@ public CreateErasureCodingZoneResponseProto createErasureCodingZone( RpcController controller, CreateErasureCodingZoneRequestProto req) throws ServiceException { try { - ECSchema schema = req.hasSchema() ? PBHelper.convertECSchema(req - .getSchema()) : null; - int cellSize = req.hasCellSize() ? req.getCellSize() : 0; - server.createErasureCodingZone(req.getSrc(), schema, cellSize); + ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(req + .getEcPolicy()) : null; + server.createErasureCodingZone(req.getSrc(), ecPolicy); return CreateErasureCodingZoneResponseProto.newBuilder().build(); } catch (IOException e) { throw new ServiceException(e); @@ -1522,14 +1521,14 @@ public GetEditsFromTxidResponseProto getEditsFromTxid(RpcController controller, } @Override - public GetECSchemasResponseProto getECSchemas(RpcController controller, - GetECSchemasRequestProto request) throws ServiceException { + public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcController controller, + GetErasureCodingPoliciesRequestProto request) throws ServiceException { try { - ECSchema[] ecSchemas = server.getECSchemas(); - GetECSchemasResponseProto.Builder resBuilder = GetECSchemasResponseProto + ErasureCodingPolicy[] ecPolicies = server.getErasureCodingPolicies(); + GetErasureCodingPoliciesResponseProto.Builder resBuilder = GetErasureCodingPoliciesResponseProto .newBuilder(); - for (ECSchema ecSchema : ecSchemas) { - resBuilder.addSchemas(PBHelper.convertECSchema(ecSchema)); + for (ErasureCodingPolicy ecPolicy : ecPolicies) { + resBuilder.addEcPolicies(PBHelper.convertErasureCodingPolicy(ecPolicy)); } return resBuilder.build(); } catch (IOException 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 2c88a93e3d55b..417ff028a3d57 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 @@ -165,12 +165,12 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; @@ -182,7 +182,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolMetaInterface; import org.apache.hadoop.ipc.ProtocolTranslator; @@ -240,8 +240,8 @@ public class ClientNamenodeProtocolTranslatorPB implements VOID_GET_STORAGE_POLICIES_REQUEST = GetStoragePoliciesRequestProto.newBuilder().build(); - private final static GetECSchemasRequestProto - VOID_GET_ECSCHEMAS_REQUEST = GetECSchemasRequestProto + private final static GetErasureCodingPoliciesRequestProto + VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto .newBuilder().build(); public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) { @@ -1419,16 +1419,13 @@ public BatchedEntries listEncryptionZones(long id) } @Override - public void createErasureCodingZone(String src, ECSchema schema, int cellSize) + public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) throws IOException { final CreateErasureCodingZoneRequestProto.Builder builder = CreateErasureCodingZoneRequestProto.newBuilder(); builder.setSrc(src); - if (schema != null) { - builder.setSchema(PBHelper.convertECSchema(schema)); - } - if (cellSize > 0) { - builder.setCellSize(cellSize); + if (ecPolicy != null) { + builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy)); } CreateErasureCodingZoneRequestProto req = builder.build(); try { @@ -1550,16 +1547,17 @@ public EventBatchList getEditsFromTxid(long txid) throws IOException { } @Override - public ECSchema[] getECSchemas() throws IOException { + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { try { - GetECSchemasResponseProto response = rpcProxy.getECSchemas(null, - VOID_GET_ECSCHEMAS_REQUEST); - ECSchema[] schemas = new ECSchema[response.getSchemasCount()]; + GetErasureCodingPoliciesResponseProto response = rpcProxy + .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST); + ErasureCodingPolicy[] ecPolicies = + new ErasureCodingPolicy[response.getEcPoliciesCount()]; int i = 0; - for (ECSchemaProto schemaProto : response.getSchemasList()) { - schemas[i++] = PBHelper.convertECSchema(schemaProto); + for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) { + ecPolicies[i++] = PBHelper.convertErasureCodingPolicy(ecPolicyProto); } - return schemas; + return ecPolicies; } 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 1bc096498e8da..f2facd7b8da27 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 @@ -78,6 +78,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -137,6 +138,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; 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.BlockStoragePolicyProto; @@ -1348,8 +1350,7 @@ public static LocatedBlocks convert(LocatedBlocksProto lb) { PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null, lb.getIsLastBlockComplete(), lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null, - lb.hasECSchema() ? convertECSchema(lb.getECSchema()) : null, - lb.hasStripeCellSize() ? lb.getStripeCellSize() : 0); + lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null); } public static LocatedBlocksProto convert(LocatedBlocks lb) { @@ -1365,11 +1366,8 @@ public static LocatedBlocksProto convert(LocatedBlocks lb) { if (lb.getFileEncryptionInfo() != null) { builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo())); } - if (lb.getECSchema() != null) { - builder.setECSchema(convertECSchema(lb.getECSchema())); - } - if (lb.getStripeCellSize() != 0) { - builder.setStripeCellSize(lb.getStripeCellSize()); + if (lb.getErasureCodingPolicy() != null) { + builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy())); } return builder.setFileLength(lb.getFileLength()) .setUnderConstruction(lb.isUnderConstruction()) @@ -1514,8 +1512,7 @@ public static HdfsFileStatus convert(HdfsFileStatusProto fs) { fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null, fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy() : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, - fs.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null, - fs.hasStripeCellSize() ? fs.getStripeCellSize() : 0); + fs.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(fs.getEcPolicy()) : null); } public static SnapshottableDirectoryStatus convert( @@ -1576,10 +1573,9 @@ public static HdfsFileStatusProto convert(HdfsFileStatus fs) { builder.setLocations(PBHelper.convert(locations)); } } - if(fs.getECSchema() != null) { - builder.setEcSchema(PBHelper.convertECSchema(fs.getECSchema())); + if(fs.getErasureCodingPolicy() != null) { + builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(fs.getErasureCodingPolicy())); } - builder.setStripeCellSize(fs.getStripeCellSize()); return builder.build(); } @@ -3137,13 +3133,12 @@ public static ECSchema convertECSchema(ECSchemaProto schema) { for (ECSchemaOptionEntryProto option : optionsList) { options.put(option.getKey(), option.getValue()); } - return new ECSchema(schema.getSchemaName(), schema.getCodecName(), - schema.getDataUnits(), schema.getParityUnits(), options); + return new ECSchema(schema.getCodecName(), schema.getDataUnits(), + schema.getParityUnits(), options); } public static ECSchemaProto convertECSchema(ECSchema schema) { ECSchemaProto.Builder builder = ECSchemaProto.newBuilder() - .setSchemaName(schema.getSchemaName()) .setCodecName(schema.getCodecName()) .setDataUnits(schema.getNumDataUnits()) .setParityUnits(schema.getNumParityUnits()); @@ -3155,17 +3150,34 @@ public static ECSchemaProto convertECSchema(ECSchema schema) { return builder.build(); } + public static ErasureCodingPolicy convertErasureCodingPolicy( + ErasureCodingPolicyProto policy) { + return new ErasureCodingPolicy(policy.getName(), + convertECSchema(policy.getSchema()), + policy.getCellSize()); + } + + public static ErasureCodingPolicyProto convertErasureCodingPolicy( + ErasureCodingPolicy policy) { + ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto + .newBuilder() + .setName(policy.getName()) + .setSchema(convertECSchema(policy.getSchema())) + .setCellSize(policy.getCellSize()); + return builder.build(); + } + public static ErasureCodingZoneProto convertErasureCodingZone( ErasureCodingZone ecZone) { return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir()) - .setSchema(convertECSchema(ecZone.getSchema())) - .setCellSize(ecZone.getCellSize()).build(); + .setEcPolicy(convertErasureCodingPolicy(ecZone.getErasureCodingPolicy())) + .build(); } public static ErasureCodingZone convertErasureCodingZone( ErasureCodingZoneProto ecZoneProto) { return new ErasureCodingZone(ecZoneProto.getDir(), - convertECSchema(ecZoneProto.getSchema()), ecZoneProto.getCellSize()); + convertErasureCodingPolicy(ecZoneProto.getEcPolicy())); } public static BlockECRecoveryInfo convertBlockECRecoveryInfo( @@ -3198,12 +3210,11 @@ public static BlockECRecoveryInfo convertBlockECRecoveryInfo( liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue(); } - ECSchema ecSchema = convertECSchema(blockEcRecoveryInfoProto.getEcSchema()); - int cellSize = blockEcRecoveryInfoProto.getCellSize(); + ErasureCodingPolicy ecPolicy = + convertErasureCodingPolicy(blockEcRecoveryInfoProto.getEcPolicy()); return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos, - targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema, - cellSize); + targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy); } public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( @@ -3228,8 +3239,8 @@ public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices(); builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices)); - builder.setEcSchema(convertECSchema(blockEcRecoveryInfo.getECSchema())); - builder.setCellSize(blockEcRecoveryInfo.getCellSize()); + builder.setEcPolicy(convertErasureCodingPolicy(blockEcRecoveryInfo + .getErasureCodingPolicy())); return builder.build(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index f101dd01ba8f9..b88b554fdcea4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -20,7 +20,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @@ -38,8 +38,7 @@ * array to record the block index for each triplet. */ public class BlockInfoStriped extends BlockInfo { - private final ECSchema schema; - private final int cellSize; + private final ErasureCodingPolicy ecPolicy; /** * Always the same size with triplets. Record the block index for each triplet * TODO: actually this is only necessary for over-replicated block. Thus can @@ -47,36 +46,34 @@ public class BlockInfoStriped extends BlockInfo { */ private byte[] indices; - public BlockInfoStriped(Block blk, ECSchema schema, int cellSize) { - super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits())); - indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()]; + public BlockInfoStriped(Block blk, ErasureCodingPolicy ecPolicy) { + super(blk, (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits())); + indices = new byte[ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()]; initIndices(); - this.schema = schema; - this.cellSize = cellSize; + this.ecPolicy = ecPolicy; } BlockInfoStriped(BlockInfoStriped b) { - this(b, b.getSchema(), b.getCellSize()); + this(b, b.getErasureCodingPolicy()); this.setBlockCollection(b.getBlockCollection()); } public short getTotalBlockNum() { - return (short) (this.schema.getNumDataUnits() - + this.schema.getNumParityUnits()); + return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()); } public short getDataBlockNum() { - return (short) this.schema.getNumDataUnits(); + return (short) ecPolicy.getNumDataUnits(); } public short getParityBlockNum() { - return (short) this.schema.getNumParityUnits(); + return (short) ecPolicy.getNumParityUnits(); } /** * If the block is committed/completed and its length is less than a full * stripe, it returns the the number of actual data blocks. - * Otherwise it returns the number of data units specified by schema. + * Otherwise it returns the number of data units specified by erasure coding policy. */ public short getRealDataBlockNum() { if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) { @@ -91,12 +88,8 @@ public short getRealTotalBlockNum() { return (short) (getRealDataBlockNum() + getParityBlockNum()); } - public ECSchema getSchema() { - return schema; - } - - public int getCellSize() { - return cellSize; + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; } private void initIndices() { @@ -230,7 +223,7 @@ public long spaceConsumed() { // be the total of data blocks and parity blocks because // `getNumBytes` is the total of actual data block size. return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(), - this.schema.getNumDataUnits(), this.schema.getNumParityUnits(), + ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(), BLOCK_STRIPED_CELL_SIZE); } @@ -260,7 +253,7 @@ public BlockInfoUnderConstructionStriped convertToBlockUnderConstruction( BlockUCState s, DatanodeStorageInfo[] targets) { final BlockInfoUnderConstructionStriped ucBlock; if(isComplete()) { - ucBlock = new BlockInfoUnderConstructionStriped(this, schema, cellSize, + ucBlock = new BlockInfoUnderConstructionStriped(this, ecPolicy, s, targets); ucBlock.setBlockCollection(getBlockCollection()); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java index 2746eeb26bf45..7df0428706117 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionStriped.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.namenode.NameNode; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import java.io.IOException; @@ -57,17 +57,16 @@ public class BlockInfoUnderConstructionStriped extends BlockInfoStriped /** * Constructor with null storage targets. */ - public BlockInfoUnderConstructionStriped(Block blk, ECSchema schema, - int cellSize) { - this(blk, schema, cellSize, UNDER_CONSTRUCTION, null); + public BlockInfoUnderConstructionStriped(Block blk, ErasureCodingPolicy ecPolicy) { + this(blk, ecPolicy, UNDER_CONSTRUCTION, null); } /** * Create a striped block that is currently being constructed. */ - public BlockInfoUnderConstructionStriped(Block blk, ECSchema schema, - int cellSize, BlockUCState state, DatanodeStorageInfo[] targets) { - super(blk, schema, cellSize); + public BlockInfoUnderConstructionStriped(Block blk, ErasureCodingPolicy ecPolicy, + BlockUCState state, DatanodeStorageInfo[] targets) { + super(blk, ecPolicy); assert getBlockUCState() != COMPLETE : "BlockInfoUnderConstructionStriped cannot be in COMPLETE state"; this.blockUCState = state; 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 0ceb3921fb90c..36ce13330997c 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 @@ -85,7 +85,7 @@ 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.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; @@ -948,14 +948,13 @@ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, ErasureCodingZone ecZone) throws IOException { assert namesystem.hasReadLock(); - final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; - final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; + final ErasureCodingPolicy ecPolicy = ecZone != null ? ecZone + .getErasureCodingPolicy() : null; if (blocks == null) { return null; } else if (blocks.length == 0) { return new LocatedBlocks(0, isFileUnderConstruction, - Collections. emptyList(), null, false, feInfo, schema, - cellSize); + Collections. emptyList(), null, false, feInfo, ecPolicy); } else { if (LOG.isDebugEnabled()) { LOG.debug("blocks = " + java.util.Arrays.asList(blocks)); @@ -980,7 +979,7 @@ Collections. emptyList(), null, false, feInfo, schema, } return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo, - schema, cellSize); + ecPolicy); } } @@ -1597,7 +1596,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { .warn("Failed to get the EC zone for the file {} ", src); } if (ecZone == null) { - blockLog.warn("No EC schema found for the file {}. " + blockLog.warn("No erasure coding policy found for the file {}. " + "So cannot proceed for recovery", src); // TODO: we may have to revisit later for what we can do better to // handle this case. @@ -1607,7 +1606,7 @@ int computeRecoveryWorkForBlocks(List> blocksToRecover) { new ExtendedBlock(namesystem.getBlockPoolId(), block), rw.srcNodes, rw.targets, ((ErasureCodingWork) rw).liveBlockIndicies, - ecZone.getSchema(), ecZone.getCellSize()); + ecZone.getErasureCodingPolicy()); } else { rw.srcNodes[0].addBlockToBeReplicated(block, targets); } 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 e4366c91161f4..108ce2f59004c 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 @@ -50,7 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; import org.apache.hadoop.hdfs.util.EnumCounters; import org.apache.hadoop.hdfs.util.LightWeightHashSet; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.util.IntrusiveCollection; import org.apache.hadoop.util.Time; @@ -610,10 +610,10 @@ void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { */ void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets, - short[] liveBlockIndices, ECSchema ecSchema, int cellSize) { + short[] liveBlockIndices, ErasureCodingPolicy ecPolicy) { assert (block != null && sources != null && sources.length > 0); BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets, - liveBlockIndices, ecSchema, cellSize); + liveBlockIndices, ecPolicy); erasurecodeBlocks.offer(task); BlockManager.LOG.debug("Adding block recovery task " + task + "to " + getName() + ", current queue size is " + erasurecodeBlocks.size()); 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 index 126086f3973fa..da89dac3ac694 100644 --- 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 @@ -54,7 +54,7 @@ private StorageLocation(StorageType storageType, URI uri) { // 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); + throw new IllegalArgumentException("Unsupported URI ecPolicy in " + uri); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 3c9adc449b2f3..f6a5ece26557f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -69,8 +69,7 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.erasurecode.ECSchema; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.token.Token; @@ -267,10 +266,10 @@ private class ReconstructAndTransferBlock implements Runnable { new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL); ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) { - ECSchema schema = recoveryInfo.getECSchema(); - dataBlkNum = schema.getNumDataUnits(); - parityBlkNum = schema.getNumParityUnits(); - cellSize = recoveryInfo.getCellSize(); + ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy(); + dataBlkNum = ecPolicy.getNumDataUnits(); + parityBlkNum = ecPolicy.getNumParityUnits(); + cellSize = ecPolicy.getCellSize(); blockGroup = recoveryInfo.getExtendedBlock(); final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java index dcab0751d8c55..dc6acd57c14c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; @@ -177,7 +177,7 @@ private ExitStatus run() { } DBlock newDBlock(LocatedBlock lb, List locations, - ECSchema ecSchema) { + ErasureCodingPolicy ecPolicy) { Block blk = lb.getBlock().getLocalBlock(); DBlock db; if (lb.isStriped()) { @@ -186,7 +186,7 @@ DBlock newDBlock(LocatedBlock lb, List locations, for (int i = 0; i < indices.length; i++) { indices[i] = (byte) lsb.getBlockIndices()[i]; } - db = new DBlockStriped(blk, indices, (short) ecSchema.getNumDataUnits()); + db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits()); } else { db = new DBlock(blk); } @@ -373,7 +373,7 @@ private boolean processFile(String fullPath, HdfsLocatedFileStatus status) { List types = policy.chooseStorageTypes( status.getReplication()); - final ECSchema ecSchema = status.getECSchema(); + final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy(); final LocatedBlocks locatedBlocks = status.getBlockLocations(); boolean hasRemaining = false; final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete(); @@ -390,7 +390,7 @@ private boolean processFile(String fullPath, HdfsLocatedFileStatus status) { final StorageTypeDiff diff = new StorageTypeDiff(types, lb.getStorageTypes()); if (!diff.removeOverlap(true)) { - if (scheduleMoves4Block(diff, lb, ecSchema)) { + if (scheduleMoves4Block(diff, lb, ecPolicy)) { hasRemaining |= (diff.existing.size() > 1 && diff.expected.size() > 1); } @@ -400,12 +400,12 @@ private boolean processFile(String fullPath, HdfsLocatedFileStatus status) { } boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb, - ECSchema ecSchema) { + ErasureCodingPolicy ecPolicy) { final List locations = MLocation.toLocations(lb); if (!(lb instanceof LocatedStripedBlock)) { Collections.shuffle(locations); } - final DBlock db = newDBlock(lb, locations, ecSchema); + final DBlock db = newDBlock(lb, locations, ecPolicy); for (final StorageType t : diff.existing) { for (final MLocation ml : locations) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java new file mode 100644 index 0000000000000..71ac36a79bbd9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.ECSchema; + +import java.util.Map; +import java.util.TreeMap; + +/** + * This manages erasure coding policies predefined and activated in the system. + * It loads customized policies and syncs with persisted ones in + * NameNode image. + * + * This class is instantiated by the FSNamesystem. + */ +@InterfaceAudience.LimitedPrivate({"HDFS"}) +public final class ErasureCodingPolicyManager { + + /** + * TODO: HDFS-8095 + */ + private static final int DEFAULT_DATA_BLOCKS = 6; + private static final int DEFAULT_PARITY_BLOCKS = 3; + private static final int DEFAULT_CELLSIZE = 64 * 1024; + private static final String DEFAULT_CODEC_NAME = "rs"; + private static final String DEFAULT_POLICY_NAME = "RS-6-3-64k"; + private static final ECSchema SYS_DEFAULT_SCHEMA = new ECSchema( + DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS); + private static final ErasureCodingPolicy SYS_DEFAULT_POLICY = + new ErasureCodingPolicy(DEFAULT_POLICY_NAME, SYS_DEFAULT_SCHEMA, + DEFAULT_CELLSIZE); + + //We may add more later. + private static ErasureCodingPolicy[] SYS_POLICY = new ErasureCodingPolicy[] { + SYS_DEFAULT_POLICY + }; + + /** + * All active policies maintained in NN memory for fast querying, + * identified and sorted by its name. + */ + private final Map activePolicies; + + ErasureCodingPolicyManager() { + + this.activePolicies = new TreeMap<>(); + for (ErasureCodingPolicy policy : SYS_POLICY) { + activePolicies.put(policy.getName(), policy); + } + + /** + * TODO: HDFS-7859 persist into NameNode + * load persistent policies from image and editlog, which is done only once + * during NameNode startup. This can be done here or in a separate method. + */ + } + + /** + * Get system defined policies. + * @return system policies + */ + public static ErasureCodingPolicy[] getSystemPolices() { + return SYS_POLICY; + } + + /** + * Get system-wide default policy, which can be used by default + * when no policy is specified for an EC zone. + * @return ecPolicy + */ + public static ErasureCodingPolicy getSystemDefaultPolicy() { + return SYS_DEFAULT_POLICY; + } + + /** + * Get all policies that's available to use. + * @return all policies + */ + public ErasureCodingPolicy[] getPolicies() { + ErasureCodingPolicy[] results = new ErasureCodingPolicy[activePolicies.size()]; + return activePolicies.values().toArray(results); + } + + /** + * Get the policy specified by the policy name. + */ + public ErasureCodingPolicy getPolicy(String name) { + return activePolicies.get(name); + } + + /** + * Clear and clean up + */ + public void clear() { + activePolicies.clear(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java deleted file mode 100644 index 4c4aae9718ad1..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java +++ /dev/null @@ -1,127 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hdfs.server.namenode; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.io.erasurecode.ECSchema; - -import java.util.Map; -import java.util.TreeMap; - -/** - * This manages EC schemas predefined and activated in the system. - * It loads customized schemas and syncs with persisted ones in - * NameNode image. - * - * This class is instantiated by the FSNamesystem. - */ -@InterfaceAudience.LimitedPrivate({"HDFS"}) -public final class ErasureCodingSchemaManager { - - /** - * TODO: HDFS-8095 - */ - private static final int DEFAULT_DATA_BLOCKS = 6; - private static final int DEFAULT_PARITY_BLOCKS = 3; - private static final String DEFAULT_CODEC_NAME = "rs"; - private static final String DEFAULT_SCHEMA_NAME = "RS-6-3"; - private static final ECSchema SYS_DEFAULT_SCHEMA = - new ECSchema(DEFAULT_SCHEMA_NAME, - DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS); - - //We may add more later. - private static ECSchema[] SYS_SCHEMAS = new ECSchema[] { - SYS_DEFAULT_SCHEMA - }; - - /** - * All active EC activeSchemas maintained in NN memory for fast querying, - * identified and sorted by its name. - */ - private final Map activeSchemas; - - ErasureCodingSchemaManager() { - - this.activeSchemas = new TreeMap(); - for (ECSchema schema : SYS_SCHEMAS) { - activeSchemas.put(schema.getSchemaName(), schema); - } - - /** - * TODO: HDFS-7859 persist into NameNode - * load persistent schemas from image and editlog, which is done only once - * during NameNode startup. This can be done here or in a separate method. - */ - } - - /** - * Get system defined schemas. - * @return system schemas - */ - public static ECSchema[] getSystemSchemas() { - return SYS_SCHEMAS; - } - - /** - * Get system-wide default EC schema, which can be used by default when no - * schema is specified for an EC zone. - * @return schema - */ - public static ECSchema getSystemDefaultSchema() { - return SYS_DEFAULT_SCHEMA; - } - - /** - * Tell the specified schema is the system default one or not. - * @param schema - * @return true if it's the default false otherwise - */ - public static boolean isSystemDefault(ECSchema schema) { - if (schema == null) { - throw new IllegalArgumentException("Invalid schema parameter"); - } - - // schema name is the identifier. - return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName()); - } - - /** - * Get all EC schemas that's available to use. - * @return all EC schemas - */ - public ECSchema[] getSchemas() { - ECSchema[] results = new ECSchema[activeSchemas.size()]; - return activeSchemas.values().toArray(results); - } - - /** - * Get the EC schema specified by the schema name. - * @param schemaName - * @return EC schema specified by the schema name - */ - public ECSchema getSchema(String schemaName) { - return activeSchemas.get(schemaName); - } - - /** - * Clear and clean up - */ - public void clear() { - activeSchemas.clear(); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java index 263812622c107..22d821fd988c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java @@ -23,11 +23,10 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.hdfs.XAttrHelper; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.WritableUtils; -import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -60,9 +59,9 @@ public ErasureCodingZoneManager(FSDirectory dir) { this.dir = dir; } - ECSchema getErasureCodingSchema(INodesInPath iip) throws IOException { + ErasureCodingPolicy getErasureCodingPolicy(INodesInPath iip) throws IOException { ErasureCodingZone ecZone = getErasureCodingZone(iip); - return ecZone == null ? null : ecZone.getSchema(); + return ecZone == null ? null : ecZone.getErasureCodingPolicy(); } ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException { @@ -88,12 +87,11 @@ ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException { if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) { ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue()); DataInputStream dIn=new DataInputStream(bIn); - int cellSize = WritableUtils.readVInt(dIn); - String schemaName = WritableUtils.readString(dIn); - ECSchema schema = dir.getFSNamesystem() - .getErasureCodingSchemaManager().getSchema(schemaName); + String ecPolicyName = WritableUtils.readString(dIn); + ErasureCodingPolicy ecPolicy = dir.getFSNamesystem() + .getErasureCodingPolicyManager().getPolicy(ecPolicyName); return new ErasureCodingZone(dir.getInode(inode.getId()) - .getFullPathName(), schema, cellSize); + .getFullPathName(), ecPolicy); } } } @@ -101,7 +99,7 @@ ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException { } List createErasureCodingZone(final INodesInPath srcIIP, - ECSchema schema, int cellSize) throws IOException { + ErasureCodingPolicy ecPolicy) throws IOException { assert dir.hasWriteLock(); Preconditions.checkNotNull(srcIIP, "INodes cannot be null"); String src = srcIIP.getPath(); @@ -115,29 +113,22 @@ List createErasureCodingZone(final INodesInPath srcIIP, throw new IOException("Attempt to create an erasure coding zone " + "for a file " + src); } - if (getErasureCodingSchema(srcIIP) != null) { + if (getErasureCodingPolicy(srcIIP) != null) { throw new IOException("Directory " + src + " is already in an " + "erasure coding zone."); } - // System default schema will be used since no specified. - if (schema == null) { - schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); + // System default erasure coding policy will be used since no specified. + if (ecPolicy == null) { + ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); } - if (cellSize <= 0) { - cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - } - - // Write the cellsize first and then schema name final XAttr ecXAttr; DataOutputStream dOut = null; try { ByteArrayOutputStream bOut = new ByteArrayOutputStream(); dOut = new DataOutputStream(bOut); - WritableUtils.writeVInt(dOut, cellSize); - // Now persist the schema name in xattr - WritableUtils.writeString(dOut, schema.getSchemaName()); + WritableUtils.writeString(dOut, ecPolicy.getName()); ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE, bOut.toByteArray()); } finally { @@ -158,10 +149,12 @@ void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src) if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) { return; } - final ECSchema srcSchema = (srcZone != null) ? srcZone.getSchema() : null; - final ECSchema dstSchema = (dstZone != null) ? dstZone.getSchema() : null; - if ((srcSchema != null && !srcSchema.equals(dstSchema)) || - (dstSchema != null && !dstSchema.equals(srcSchema))) { + final ErasureCodingPolicy srcECPolicy = + srcZone != null ? srcZone.getErasureCodingPolicy() : null; + final ErasureCodingPolicy dstECPolicy = + dstZone != null ? dstZone.getErasureCodingPolicy() : null; + if (srcECPolicy != null && !srcECPolicy.equals(dstECPolicy) || + dstECPolicy != null && !dstECPolicy.equals(srcECPolicy)) { throw new IOException( src + " can't be moved because the source and destination have " + "different erasure coding policies."); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java index fd7ef333cdb4c..8c515d1641613 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java @@ -22,9 +22,9 @@ import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; -import org.apache.hadoop.io.erasurecode.ECSchema; /** * Helper class to perform erasure coding related operations. @@ -43,15 +43,14 @@ private FSDirErasureCodingOp() {} * @param fsn namespace * @param srcArg the path of a directory which will be the root of the * erasure coding zone. The directory must be empty. - * @param schema ECSchema for the erasure coding zone - * @param cellSize Cell size of stripe + * @param ecPolicy erasure coding policy for the erasure coding zone * @param logRetryCache whether to record RPC ids in editlog for retry * cache rebuilding * @return {@link HdfsFileStatus} * @throws IOException */ static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn, - final String srcArg, final ECSchema schema, final int cellSize, + final String srcArg, final ErasureCodingPolicy ecPolicy, final boolean logRetryCache) throws IOException { assert fsn.hasWriteLock(); @@ -68,7 +67,7 @@ static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn, try { iip = fsd.getINodesInPath4Write(src, false); xAttrs = fsn.getErasureCodingZoneManager().createErasureCodingZone( - iip, schema, cellSize); + iip, ecPolicy); } finally { fsd.writeUnlock(); } @@ -120,7 +119,7 @@ static boolean isInErasureCodingZone(final FSNamesystem fsn, assert fsn.hasReadLock(); final INodesInPath iip = getINodesInPath(fsn, srcArg); - return getErasureCodingSchemaForPath(fsn, iip) != null; + return getErasureCodingPolicyForPath(fsn, iip) != null; } /** @@ -133,49 +132,35 @@ static boolean isInErasureCodingZone(final FSNamesystem fsn, */ static boolean isInErasureCodingZone(final FSNamesystem fsn, final INodesInPath iip) throws IOException { - return getErasureCodingSchema(fsn, iip) != null; + return getErasureCodingPolicy(fsn, iip) != null; } /** - * Get erasure coding schema. + * Get the erasure coding policy. * * @param fsn namespace * @param iip inodes in the path containing the file - * @return {@link ECSchema} + * @return {@link ErasureCodingPolicy} * @throws IOException */ - static ECSchema getErasureCodingSchema(final FSNamesystem fsn, + static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn, final INodesInPath iip) throws IOException { assert fsn.hasReadLock(); - return getErasureCodingSchemaForPath(fsn, iip); + return getErasureCodingPolicyForPath(fsn, iip); } /** - * Get available erasure coding schemas. + * Get available erasure coding polices. * * @param fsn namespace - * @return {@link ECSchema} array + * @return {@link ErasureCodingPolicy} array */ - static ECSchema[] getErasureCodingSchemas(final FSNamesystem fsn) + static ErasureCodingPolicy[] getErasureCodingPolicies(final FSNamesystem fsn) throws IOException { assert fsn.hasReadLock(); - return fsn.getErasureCodingSchemaManager().getSchemas(); - } - - /** - * Get the ECSchema specified by the name. - * - * @param fsn namespace - * @param schemaName schema name - * @return {@link ECSchema} - */ - static ECSchema getErasureCodingSchema(final FSNamesystem fsn, - final String schemaName) throws IOException { - assert fsn.hasReadLock(); - - return fsn.getErasureCodingSchemaManager().getSchema(schemaName); + return fsn.getErasureCodingPolicyManager().getPolicies(); } private static INodesInPath getINodesInPath(final FSNamesystem fsn, @@ -204,12 +189,12 @@ private static ErasureCodingZone getErasureCodingZoneForPath( } } - private static ECSchema getErasureCodingSchemaForPath(final FSNamesystem fsn, + private static ErasureCodingPolicy getErasureCodingPolicyForPath(final FSNamesystem fsn, final INodesInPath iip) throws IOException { final FSDirectory fsd = fsn.getFSDirectory(); fsd.readLock(); try { - return fsn.getErasureCodingZoneManager().getErasureCodingSchema(iip); + return fsn.getErasureCodingZoneManager().getErasureCodingPolicy(iip); } finally { fsd.readUnlock(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index 1e4ee83f8dc38..bad7c423115f8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -40,7 +41,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.util.ReadOnlyList; -import org.apache.hadoop.io.erasurecode.ECSchema; import java.io.FileNotFoundException; import java.io.IOException; @@ -318,7 +318,7 @@ static HdfsFileStatus getFileInfo( if (fsd.getINode4DotSnapshot(srcs) != null) { return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null, HdfsFileStatus.EMPTY_NAME, -1L, 0, null, - HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0); + HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); } return null; } @@ -388,8 +388,8 @@ static HdfsFileStatus createFileStatus( final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsd.getFSNamesystem(), iip); - final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; - final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; + final ErasureCodingPolicy ecPolicy = + ecZone != null ? ecZone.getErasureCodingPolicy() : null; if (node.isFile()) { final INodeFile fileNode = node.asFile(); @@ -421,8 +421,7 @@ static HdfsFileStatus createFileStatus( childrenNum, feInfo, storagePolicy, - schema, - cellSize); + ecPolicy); } private static INodeAttributes getINodeAttributes( @@ -471,8 +470,8 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; - final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; + final ErasureCodingPolicy ecPolicy = + ecZone != null ? ecZone.getErasureCodingPolicy() : null; HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(size, node.isDirectory(), replication, @@ -481,8 +480,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( getPermissionForFileStatus(nodeAttrs, isEncrypted), nodeAttrs.getUserName(), nodeAttrs.getGroupName(), node.isSymlink() ? node.asSymlink().getSymlink() : null, path, - node.getId(), loc, childrenNum, feInfo, storagePolicy, schema, - cellSize); + node.getId(), loc, childrenNum, feInfo, storagePolicy, ecPolicy); // Set caching information for the located blocks. if (loc != null) { CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 254e850e41bda..8f4f51ad59959 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -51,7 +52,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.net.Node; import org.apache.hadoop.net.NodeBase; import org.apache.hadoop.util.ChunkedArrayList; @@ -532,16 +532,15 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, if (isStriped) { ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( fsd.getFSNamesystem(), inodesInPath); - ECSchema ecSchema = ecZone.getSchema(); - short numDataUnits = (short) ecSchema.getNumDataUnits(); - short numParityUnits = (short) ecSchema.getNumParityUnits(); + ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy(); + short numDataUnits = (short) ecPolicy.getNumDataUnits(); + short numParityUnits = (short) ecPolicy.getNumParityUnits(); short numLocations = (short) (numDataUnits + numParityUnits); // check quota limits and updated space consumed fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), numLocations, true); - blockInfo = new BlockInfoUnderConstructionStriped(block, ecSchema, - ecZone.getCellSize(), + blockInfo = new BlockInfoUnderConstructionStriped(block, ecPolicy, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } else { // check quota limits and updated space consumed 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 a1f38a35c9878..b5b7178fe2a0f 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 @@ -992,7 +992,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped = ecZone != null; if (isStriped) { newBlockInfo = new BlockInfoUnderConstructionStriped(newBlock, - ecZone.getSchema(), ecZone.getCellSize()); + ecZone.getErasureCodingPolicy()); } else { newBlockInfo = new BlockInfoUnderConstructionContiguous(newBlock, file.getPreferredBlockReplication()); @@ -1078,7 +1078,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // until several blocks in? if (isStriped) { newBI = new BlockInfoUnderConstructionStriped(newBlock, - ecZone.getSchema(), ecZone.getCellSize()); + ecZone.getErasureCodingPolicy()); } else { newBI = new BlockInfoUnderConstructionContiguous(newBlock, file.getPreferredBlockReplication()); @@ -1088,11 +1088,9 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // is only executed when loading edits written by prior // versions of Hadoop. Current versions always log // OP_ADD operations as each block is allocated. - // TODO: ECSchema can be restored from persisted file (HDFS-7859). if (isStriped) { newBI = new BlockInfoStriped(newBlock, - ErasureCodingSchemaManager.getSystemDefaultSchema(), - ecZone.getCellSize()); + ErasureCodingPolicyManager.getSystemDefaultPolicy()); } else { newBI = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); 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 3efb9338de396..653bd4a25cd87 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; @@ -66,7 +67,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.protobuf.ByteString; -import org.apache.hadoop.io.erasurecode.ECSchema; @InterfaceAudience.Private public final class FSImageFormatPBINode { @@ -327,17 +327,13 @@ private INodeFile loadINodeFile(INodeSection.INode n) { short replication = (short) f.getReplication(); boolean isStriped = f.getIsStriped(); LoaderContext state = parent.getLoaderContext(); - ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); + ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); - if (isStriped) { - Preconditions.checkState(f.hasStripingCellSize()); - } BlockInfo[] blocks = new BlockInfo[bp.size()]; for (int i = 0; i < bp.size(); ++i) { BlockProto b = bp.get(i); if (isStriped) { - blocks[i] = new BlockInfoStriped(PBHelper.convert(b), schema, - (int)f.getStripingCellSize()); + blocks[i] = new BlockInfoStriped(PBHelper.convert(b), ecPolicy); } else { blocks[i] = new BlockInfoContiguous(PBHelper.convert(b), replication); @@ -373,8 +369,7 @@ private INodeFile loadINodeFile(INodeSection.INode n) { final BlockInfo ucBlk; if (isStriped) { BlockInfoStriped striped = (BlockInfoStriped) lastBlk; - ucBlk = new BlockInfoUnderConstructionStriped(striped, - schema, (int)f.getStripingCellSize()); + ucBlk = new BlockInfoUnderConstructionStriped(striped, ecPolicy); } else { ucBlk = new BlockInfoUnderConstructionContiguous(lastBlk, replication); @@ -656,16 +651,6 @@ private void save(OutputStream out, INodeFile n) throws IOException { } } - if (n.isStriped()) { - if (blocks != null && blocks.length > 0) { - BlockInfo firstBlock = blocks[0]; - Preconditions.checkState(firstBlock.isStriped()); - b.setStripingCellSize(((BlockInfoStriped)firstBlock).getCellSize()); - } else { - b.setStripingCellSize(HdfsConstants.BLOCK_STRIPED_CELL_SIZE); - } - } - FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature(); if (uc != null) { INodeSection.FileUnderConstructionFeature f = 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 541bfa4626ed0..657f29d628049 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 @@ -180,6 +180,7 @@ 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.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -262,7 +263,6 @@ import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.RetriableException; import org.apache.hadoop.ipc.RetryCache; import org.apache.hadoop.ipc.Server; @@ -426,7 +426,7 @@ private void logAuditEvent(boolean succeeded, private final BlockManager blockManager; private final SnapshotManager snapshotManager; private final CacheManager cacheManager; - private final ErasureCodingSchemaManager ecSchemaManager; + private final ErasureCodingPolicyManager ecPolicyManager; private final DatanodeStatistics datanodeStatistics; private String nameserviceId; @@ -606,7 +606,7 @@ void clear() { leaseManager.removeAllLeases(); snapshotManager.clearSnapshottableDirs(); cacheManager.clear(); - ecSchemaManager.clear(); + ecPolicyManager.clear(); setImageLoaded(false); blockManager.clear(); } @@ -846,7 +846,7 @@ static FSNamesystem loadFromDisk(Configuration conf) throws IOException { this.dir = new FSDirectory(this, conf); this.snapshotManager = new SnapshotManager(dir); this.cacheManager = new CacheManager(this, conf, blockManager); - this.ecSchemaManager = new ErasureCodingSchemaManager(); + this.ecPolicyManager = new ErasureCodingPolicyManager(); this.safeMode = new SafeModeInfo(conf); this.topConf = new TopConf(conf); this.auditLoggers = initAuditLoggers(conf); @@ -3679,16 +3679,16 @@ void commitOrCompleteLastBlock( if (fileINode.isStriped()) { final ErasureCodingZone ecZone = FSDirErasureCodingOp .getErasureCodingZone(this, iip); - final ECSchema ecSchema = ecZone.getSchema(); - final short numDataUnits = (short) ecSchema.getNumDataUnits(); - final short numParityUnits = (short) ecSchema.getNumParityUnits(); + final ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy(); + final short numDataUnits = (short) ecPolicy.getNumDataUnits(); + final short numParityUnits = (short) ecPolicy.getNumParityUnits(); final long numBlocks = numDataUnits + numParityUnits; final long fullBlockGroupSize = fileINode.getPreferredBlockSize() * numBlocks; final BlockInfoStriped striped = new BlockInfoStriped(commitBlock, - ecSchema, ecZone.getCellSize()); + ecPolicy); final long actualBlockGroupSize = striped.spaceConsumed(); diff = fullBlockGroupSize - actualBlockGroupSize; @@ -6676,9 +6676,9 @@ public CacheManager getCacheManager() { return cacheManager; } - /** @return the ErasureCodingSchemaManager. */ - public ErasureCodingSchemaManager getErasureCodingSchemaManager() { - return ecSchemaManager; + /** @return the ErasureCodingPolicyManager. */ + public ErasureCodingPolicyManager getErasureCodingPolicyManager() { + return ecPolicyManager; } /** @return the ErasureCodingZoneManager. */ @@ -7581,14 +7581,13 @@ BatchedListEntries listEncryptionZones(long prevId) * Create an erasure coding zone on directory src. * @param srcArg the path of a directory which will be the root of the * erasure coding zone. The directory must be empty. - * @param schema ECSchema for the erasure coding zone - * @param cellSize Cell size of stripe + * @param ecPolicy erasure coding policy for the erasure coding zone * @throws AccessControlException if the caller is not the superuser. * @throws UnresolvedLinkException if the path can't be resolved. * @throws SafeModeException if the Namenode is in safe mode. */ - void createErasureCodingZone(final String srcArg, final ECSchema schema, - int cellSize, final boolean logRetryCache) throws IOException, + void createErasureCodingZone(final String srcArg, final ErasureCodingPolicy + ecPolicy, final boolean logRetryCache) throws IOException, UnresolvedLinkException, SafeModeException, AccessControlException { checkSuperuserPrivilege(); checkOperation(OperationCategory.WRITE); @@ -7599,7 +7598,7 @@ void createErasureCodingZone(final String srcArg, final ECSchema schema, checkOperation(OperationCategory.WRITE); checkNameNodeSafeMode("Cannot create erasure coding zone on " + srcArg); resultingStat = FSDirErasureCodingOp.createErasureCodingZone(this, - srcArg, schema, cellSize, logRetryCache); + srcArg, ecPolicy, logRetryCache); success = true; } finally { writeUnlock(); @@ -7627,30 +7626,15 @@ ErasureCodingZone getErasureCodingZone(String src) } /** - * Get available erasure coding schemas + * Get available erasure coding polices */ - ECSchema[] getErasureCodingSchemas() throws IOException { + ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { checkOperation(OperationCategory.READ); waitForLoadingFSImage(); readLock(); try { checkOperation(OperationCategory.READ); - return FSDirErasureCodingOp.getErasureCodingSchemas(this); - } finally { - readUnlock(); - } - } - - /** - * Get the ECSchema specified by the name - */ - ECSchema getErasureCodingSchema(String schemaName) throws IOException { - checkOperation(OperationCategory.READ); - waitForLoadingFSImage(); - readLock(); - try { - checkOperation(OperationCategory.READ); - return FSDirErasureCodingOp.getErasureCodingSchema(this, schemaName); + return FSDirErasureCodingOp.getErasureCodingPolicies(this); } finally { readUnlock(); } 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 031ae085a72d9..fffa0f04dd2db 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 @@ -84,6 +84,7 @@ 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.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -143,7 +144,6 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RetryCache; @@ -1823,7 +1823,7 @@ public BatchedEntries listEncryptionZones( } @Override // ClientProtocol - public void createErasureCodingZone(String src, ECSchema schema, int cellSize) + public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) throws IOException { checkNNStartup(); final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); @@ -1832,8 +1832,7 @@ public void createErasureCodingZone(String src, ECSchema schema, int cellSize) } boolean success = false; try { - namesystem.createErasureCodingZone(src, schema, cellSize, - cacheEntry != null); + namesystem.createErasureCodingZone(src, ecPolicy, cacheEntry != null); success = true; } finally { RetryCache.setState(cacheEntry, success); @@ -2035,9 +2034,9 @@ public void removeSpanReceiver(long id) throws IOException { } @Override // ClientProtocol - public ECSchema[] getECSchemas() throws IOException { + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { checkNNStartup(); - return namesystem.getErasureCodingSchemas(); + return namesystem.getErasureCodingPolicies(); } @Override // ClientProtocol 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 3e4a287893e72..3a2c3d475bc9b 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 @@ -572,7 +572,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res // count expected replicas short targetFileReplication; - if (file.getECSchema() != null) { + if (file.getErasureCodingPolicy() != null) { assert storedBlock instanceof BlockInfoStriped; targetFileReplication = ((BlockInfoStriped) storedBlock) .getRealTotalBlockNum(); @@ -1158,11 +1158,11 @@ public String toString() { @VisibleForTesting static class ErasureCodingResult extends Result { - final String defaultSchema; + final String defaultECPolicy; ErasureCodingResult(Configuration conf) { - defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema() - .getSchemaName(); + defaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy() + .getName(); } @Override @@ -1239,7 +1239,7 @@ public String toString() { ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks)) .append(" %)"); } - res.append("\n Default schema:\t\t").append(defaultSchema) + res.append("\n Default ecPolicy:\t\t").append(defaultECPolicy) .append("\n Average block group size:\t").append( getReplicationFactor()).append("\n Missing block groups:\t\t").append( missingIds.size()).append("\n Corrupt block groups:\t\t").append( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java index 56a1546d29966..55ae7b665a2a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import java.util.Arrays; import java.util.Collection; @@ -77,31 +77,28 @@ public static class BlockECRecoveryInfo { private String[] targetStorageIDs; private StorageType[] targetStorageTypes; private final short[] liveBlockIndices; - private final ECSchema ecSchema; - private final int cellSize; + private final ErasureCodingPolicy ecPolicy; public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices, - ECSchema ecSchema, int cellSize) { + ErasureCodingPolicy ecPolicy) { this(block, sources, DatanodeStorageInfo .toDatanodeInfos(targetDnStorageInfo), DatanodeStorageInfo .toStorageIDs(targetDnStorageInfo), DatanodeStorageInfo - .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecSchema, - cellSize); + .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecPolicy); } public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, DatanodeInfo[] targets, String[] targetStorageIDs, StorageType[] targetStorageTypes, short[] liveBlockIndices, - ECSchema ecSchema, int cellSize) { + ErasureCodingPolicy ecPolicy) { this.block = block; this.sources = sources; this.targets = targets; this.targetStorageIDs = targetStorageIDs; this.targetStorageTypes = targetStorageTypes; this.liveBlockIndices = liveBlockIndices; - this.ecSchema = ecSchema; - this.cellSize = cellSize; + this.ecPolicy = ecPolicy; } public ExtendedBlock getExtendedBlock() { @@ -128,12 +125,8 @@ public short[] getLiveBlockIndices() { return liveBlockIndices; } - public ECSchema getECSchema() { - return ecSchema; - } - - public int getCellSize() { - return cellSize; + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index 03026d895e4ef..f3260dab48bf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -31,9 +31,8 @@ import org.apache.hadoop.fs.shell.PathData; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.util.StringUtils; /** @@ -49,7 +48,7 @@ public static void registerCommands(CommandFactory factory) { factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME); factory.addClass(GetECZoneCommand.class, "-" + GetECZoneCommand.NAME); - factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME); + factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME); } @Override @@ -77,35 +76,24 @@ protected void processPath(PathData item) throws IOException { } /** - * Create EC encoding zone command. Zones are created to use specific EC - * encoding schema, other than default while encoding the files under some - * specific directory. + * A command to create an EC zone for a path, with a erasure coding policy name. */ static class CreateECZoneCommand extends ECCommand { public static final String NAME = "createZone"; - public static final String USAGE = "[-s ] [-c ] "; + public static final String USAGE = "[-s ] "; public static final String DESCRIPTION = - "Create a zone to encode files using a specified schema\n" + "Create a zone to encode files using a specified policy\n" + "Options :\n" - + " -s : EC schema name to encode files. " - + "If not passed default schema will be used\n" - + " -c : cell size to use for striped encoding files." - + " If not passed default cellsize of " - + HdfsConstants.BLOCK_STRIPED_CELL_SIZE + " will be used\n" + + " -s : erasure coding policy name to encode files. " + + "If not passed the default policy will be used\n" + " : Path to an empty directory. Under this directory " - + "files will be encoded using specified schema"; - private String schemaName; - private int cellSize = 0; - private ECSchema schema = null; + + "files will be encoded using specified erasure coding policy"; + private String ecPolicyName; + private ErasureCodingPolicy ecPolicy = null; @Override protected void processOptions(LinkedList args) throws IOException { - schemaName = StringUtils.popOptionWithArgument("-s", args); - String cellSizeStr = StringUtils.popOptionWithArgument("-c", args); - if (cellSizeStr != null) { - cellSize = (int) StringUtils.TraditionalBinaryPrefix - .string2long(cellSizeStr); - } + ecPolicyName = StringUtils.popOptionWithArgument("-s", args); if (args.isEmpty()) { throw new HadoopIllegalArgumentException(" is missing"); } @@ -119,29 +107,29 @@ protected void processPath(PathData item) throws IOException { super.processPath(item); DistributedFileSystem dfs = (DistributedFileSystem) item.fs; try { - if (schemaName != null) { - ECSchema[] ecSchemas = dfs.getClient().getECSchemas(); - for (ECSchema ecSchema : ecSchemas) { - if (schemaName.equals(ecSchema.getSchemaName())) { - schema = ecSchema; + if (ecPolicyName != null) { + ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies(); + for (ErasureCodingPolicy ecPolicy : ecPolicies) { + if (ecPolicyName.equals(ecPolicy.getName())) { + this.ecPolicy = ecPolicy; break; } } - if (schema == null) { + if (ecPolicy == null) { StringBuilder sb = new StringBuilder(); - sb.append("Schema '"); - sb.append(schemaName); - sb.append("' does not match any of the supported schemas."); + sb.append("Policy '"); + sb.append(ecPolicyName); + sb.append("' does not match any of the supported policies."); sb.append(" Please select any one of "); - List schemaNames = new ArrayList(); - for (ECSchema ecSchema : ecSchemas) { - schemaNames.add(ecSchema.getSchemaName()); + List ecPolicyNames = new ArrayList(); + for (ErasureCodingPolicy ecPolicy : ecPolicies) { + ecPolicyNames.add(ecPolicy.getName()); } - sb.append(schemaNames); + sb.append(ecPolicyNames); throw new HadoopIllegalArgumentException(sb.toString()); } } - dfs.createErasureCodingZone(item.path, schema, cellSize); + dfs.createErasureCodingZone(item.path, ecPolicy); out.println("EC Zone created successfully at " + item.path); } catch (IOException e) { throw new IOException("Unable to create EC zone for the path " @@ -188,13 +176,13 @@ protected void processPath(PathData item) throws IOException { } /** - * List all supported EC Schemas + * List all supported erasure coding policies */ - static class ListECSchemas extends ECCommand { - public static final String NAME = "listSchemas"; + static class ListPolicies extends ECCommand { + public static final String NAME = "listPolicies"; public static final String USAGE = ""; public static final String DESCRIPTION = - "Get the list of ECSchemas supported\n"; + "Get the list of erasure coding policies supported\n"; @Override protected void processOptions(LinkedList args) throws IOException { @@ -209,14 +197,14 @@ protected void processOptions(LinkedList args) throws IOException { } DistributedFileSystem dfs = (DistributedFileSystem) fs; - ECSchema[] ecSchemas = dfs.getClient().getECSchemas(); + ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies(); StringBuilder sb = new StringBuilder(); int i = 0; - while (i < ecSchemas.length) { - ECSchema ecSchema = ecSchemas[i]; - sb.append(ecSchema.getSchemaName()); + while (i < ecPolicies.length) { + ErasureCodingPolicy ecPolicy = ecPolicies[i]; + sb.append(ecPolicy.getName()); i++; - if (i < ecSchemas.length) { + if (i < ecPolicies.length) { sb.append(", "); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 4dc94a0d7ace4..4fded73c5e27c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -31,7 +31,7 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.security.token.Token; @@ -318,7 +318,7 @@ public static int convertDecodeIndexBack(int index, int dataBlkNum, } /** - * Decode based on the given input buffers and schema. + * Decode based on the given input buffers and erasure coding policy. */ public static void decodeAndFillBuffer(final byte[][] decodeInputs, AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum, @@ -355,20 +355,20 @@ public static void decodeAndFillBuffer(final byte[][] decodeInputs, * by stateful read and uses ByteBuffer as reading target buffer. Besides the * read range is within a single stripe thus the calculation logic is simpler. */ - public static AlignedStripe[] divideOneStripe(ECSchema ecSchema, + public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy, int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, long rangeEndInBlockGroup, ByteBuffer buf) { - final int dataBlkNum = ecSchema.getNumDataUnits(); + final int dataBlkNum = ecPolicy.getNumDataUnits(); // Step 1: map the byte range to StripingCells - StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize, + StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize, blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup); // Step 2: get the unmerged ranges on each internal block - VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize, + VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize, cells); // Step 3: merge into stripes - AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges); // Step 4: calculate each chunk's position in destination buffer. Since the // whole read range is within a single stripe, the logic is simpler here. @@ -400,7 +400,7 @@ public static AlignedStripe[] divideOneStripe(ECSchema ecSchema, /** * This method divides a requested byte range into an array of inclusive * {@link AlignedStripe}. - * @param ecSchema The codec schema for the file, which carries the numbers + * @param ecPolicy The codec policy for the file, which carries the numbers * of data / parity blocks * @param cellSize Cell size of stripe * @param blockGroup The striped block group @@ -412,24 +412,24 @@ public static AlignedStripe[] divideOneStripe(ECSchema ecSchema, * At most 5 stripes will be generated from each logical range, as * demonstrated in the header of {@link AlignedStripe}. */ - public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, + public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy ecPolicy, int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf, int offsetInBuf) { // Step 0: analyze range and calculate basic parameters - final int dataBlkNum = ecSchema.getNumDataUnits(); + final int dataBlkNum = ecPolicy.getNumDataUnits(); // Step 1: map the byte range to StripingCells - StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize, + StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize, blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup); // Step 2: get the unmerged ranges on each internal block - VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize, + VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize, cells); // Step 3: merge into at most 5 stripes - AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges); + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges); // Step 4: calculate each chunk's position in destination buffer calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf); @@ -446,7 +446,7 @@ public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema, * used by {@link DFSStripedOutputStream} in encoding */ @VisibleForTesting - private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, + private static StripingCell[] getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy, int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, long rangeEndInBlockGroup) { Preconditions.checkArgument( @@ -461,16 +461,16 @@ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize); final int firstCellSize = (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len); - cells[0] = new StripingCell(ecSchema, firstCellSize, firstCellIdxInBG, + cells[0] = new StripingCell(ecPolicy, firstCellSize, firstCellIdxInBG, firstCellOffset); if (lastCellIdxInBG != firstCellIdxInBG) { final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1; - cells[numCells - 1] = new StripingCell(ecSchema, lastCellSize, + cells[numCells - 1] = new StripingCell(ecPolicy, lastCellSize, lastCellIdxInBG, 0); } for (int i = 1; i < numCells - 1; i++) { - cells[i] = new StripingCell(ecSchema, cellSize, i + firstCellIdxInBG, 0); + cells[i] = new StripingCell(ecPolicy, cellSize, i + firstCellIdxInBG, 0); } return cells; @@ -481,10 +481,10 @@ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema, * the physical byte range (inclusive) on each stored internal block. */ @VisibleForTesting - private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema, + private static VerticalRange[] getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy, int cellSize, StripingCell[] cells) { - int dataBlkNum = ecSchema.getNumDataUnits(); - int parityBlkNum = ecSchema.getNumParityUnits(); + int dataBlkNum = ecPolicy.getNumDataUnits(); + int parityBlkNum = ecPolicy.getNumParityUnits(); VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum]; @@ -521,9 +521,9 @@ private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema, * {@link AlignedStripe} instances. */ private static AlignedStripe[] mergeRangesForInternalBlocks( - ECSchema ecSchema, VerticalRange[] ranges) { - int dataBlkNum = ecSchema.getNumDataUnits(); - int parityBlkNum = ecSchema.getNumParityUnits(); + ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) { + int dataBlkNum = ecPolicy.getNumDataUnits(); + int parityBlkNum = ecPolicy.getNumParityUnits(); List stripes = new ArrayList<>(); SortedSet stripePoints = new TreeSet<>(); for (VerticalRange r : ranges) { @@ -628,7 +628,7 @@ private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, */ @VisibleForTesting static class StripingCell { - final ECSchema schema; + final ErasureCodingPolicy ecPolicy; /** Logical order in a block group, used when doing I/O to a block group */ final int idxInBlkGroup; final int idxInInternalBlk; @@ -642,13 +642,13 @@ static class StripingCell { final int offset; final int size; - StripingCell(ECSchema ecSchema, int cellSize, int idxInBlkGroup, + StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup, int offset) { - this.schema = ecSchema; + this.ecPolicy = ecPolicy; this.idxInBlkGroup = idxInBlkGroup; - this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits(); + this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits(); this.idxInStripe = idxInBlkGroup - - this.idxInInternalBlk * ecSchema.getNumDataUnits(); + this.idxInInternalBlk * ecPolicy.getNumDataUnits(); this.offset = offset; this.size = cellSize; } 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 2b64ca0d4c9a5..a998147004920 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto @@ -863,8 +863,8 @@ service ClientNamenodeProtocol { returns(GetCurrentEditLogTxidResponseProto); rpc getEditsFromTxid(GetEditsFromTxidRequestProto) returns(GetEditsFromTxidResponseProto); - rpc getECSchemas(GetECSchemasRequestProto) - returns(GetECSchemasResponseProto); + rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto) + returns(GetErasureCodingPoliciesResponseProto); rpc getErasureCodingZone(GetErasureCodingZoneRequestProto) returns(GetErasureCodingZoneResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto index 56bb7a2f2961c..d27f7828f4882 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto @@ -28,24 +28,22 @@ import "hdfs.proto"; */ message ErasureCodingZoneProto { required string dir = 1; - required ECSchemaProto schema = 2; - required uint32 cellSize = 3; + required ErasureCodingPolicyProto ecPolicy = 2; } message CreateErasureCodingZoneRequestProto { required string src = 1; - optional ECSchemaProto schema = 2; - optional uint32 cellSize = 3; + optional ErasureCodingPolicyProto ecPolicy = 2; } message CreateErasureCodingZoneResponseProto { } -message GetECSchemasRequestProto { // void request +message GetErasureCodingPoliciesRequestProto { // void request } -message GetECSchemasResponseProto { - repeated ECSchemaProto schemas = 1; +message GetErasureCodingPoliciesResponseProto { + repeated ErasureCodingPolicyProto ecPolicies = 1; } message GetErasureCodingZoneRequestProto { @@ -66,6 +64,5 @@ message BlockECRecoveryInfoProto { required StorageUuidsProto targetStorageUuids = 4; required StorageTypesProto targetStorageTypes = 5; repeated uint32 liveBlockIndices = 6; - required ECSchemaProto ecSchema = 7; - required uint32 cellSize = 8; -} \ No newline at end of file + required ErasureCodingPolicyProto ecPolicy = 7; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto index 3233f66e5240f..43a1fc5fc502d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto @@ -141,7 +141,6 @@ message INodeSection { optional XAttrFeatureProto xAttrs = 9; optional uint32 storagePolicyID = 10; optional bool isStriped = 11; - optional uint64 stripingCellSize = 12; } message QuotaByStorageTypeEntryProto { 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 d2cb66538efbd..63fe90c21599a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -306,8 +306,7 @@ message LocatedBlocksProto { optional FileEncryptionInfoProto fileEncryptionInfo = 6; // Optional field for erasure coding - optional ECSchemaProto eCSchema = 7; - optional uint32 stripeCellSize = 8; + optional ErasureCodingPolicyProto ecPolicy = 7; } /** @@ -322,11 +321,16 @@ message ECSchemaOptionEntryProto { * ECSchema for erasurecoding */ message ECSchemaProto { - required string schemaName = 1; - required string codecName = 2; - required uint32 dataUnits = 3; - required uint32 parityUnits = 4; - repeated ECSchemaOptionEntryProto options = 5; + required string codecName = 1; + required uint32 dataUnits = 2; + required uint32 parityUnits = 3; + repeated ECSchemaOptionEntryProto options = 4; +} + +message ErasureCodingPolicyProto { + required string name = 1; + required ECSchemaProto schema = 2; + required uint32 cellSize = 3; } /** @@ -365,8 +369,7 @@ message HdfsFileStatusProto { optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id // Optional field for erasure coding - optional ECSchemaProto ecSchema = 17; - optional uint32 stripeCellSize = 18; + optional ErasureCodingPolicyProto ecPolicy = 17; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java index 5f01ea291641d..dfefb6676235f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java @@ -1,3 +1,4 @@ + /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file 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 0258591d2a003..28283f3c99510 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 @@ -1875,7 +1875,7 @@ public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir assert dir != null; dfs.mkdirs(dir); try { - dfs.getClient().createErasureCodingZone(dir.toString(), null, 0); + dfs.getClient().createErasureCodingZone(dir.toString(), null); } catch (IOException e) { if (!e.getMessage().contains("non-empty directory")) { throw e; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java index 55a335f988f97..df2643a1feb06 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java @@ -255,12 +255,12 @@ public Object answer(InvocationOnMock invocation) Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null, 0)).when(mockNN).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString()); Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null, 0)) + 1010, 0, null, (byte) 0, null)) .when(mockNN) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), @@ -496,7 +496,7 @@ private LocatedBlocks makeBadBlockList(LocatedBlocks goodBlockList) { badBlocks.add(badLocatedBlock); return new LocatedBlocks(goodBlockList.getFileLength(), false, badBlocks, null, true, - null, null, 0); + null, null); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index baf610654ebc9..ed3c11000dccf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -35,10 +35,10 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.junit.After; import org.junit.Assert; @@ -59,7 +59,7 @@ public class TestDFSStripedInputStream { private DistributedFileSystem fs; private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); - private final ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema(); + private final ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; @@ -79,7 +79,7 @@ public void setup() throws IOException { } fs = cluster.getFileSystem(); fs.mkdirs(dirPath); - fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE); + fs.getClient().createErasureCodingZone(dirPath.toString(), null); } @After @@ -100,7 +100,7 @@ public void testRefreshBlock() throws Exception { LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema, CELLSIZE, null); + filePath.toString(), false, ecPolicy, null); List lbList = lbs.getLocatedBlocks(); for (LocatedBlock aLbList : lbList) { @@ -152,7 +152,7 @@ public void testPread() throws Exception { } } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), - filePath.toString(), false, schema, CELLSIZE, null); + filePath.toString(), false, ecPolicy, null); int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102, CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102, @@ -194,7 +194,7 @@ public void testPreadWithDNFailure() throws Exception { } DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, - ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE, null); + ErasureCodingPolicyManager.getSystemDefaultPolicy(), null); int readSize = BLOCK_GROUP_SIZE; byte[] readBuffer = new byte[readSize]; byte[] expected = new byte[readSize]; @@ -292,7 +292,7 @@ private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), filePath.toString(), - false, schema, CELLSIZE, null); + false, ecPolicy, null); byte[] expected = new byte[fileSize]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 3f40deeae29a5..5cab978a13eea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -68,7 +68,7 @@ public void setup() throws IOException { conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index 6594ae1b47ea0..fed9f167643f0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -79,7 +79,7 @@ private void setup(Configuration conf) throws IOException { cluster.waitActive(); dfs = cluster.getFileSystem(); dfs.mkdirs(dir); - dfs.createErasureCodingZone(dir, null, 0); + dfs.createErasureCodingZone(dir, null); } private void tearDown() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index 4233a1ce238ac..6a24685e0da75 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -110,7 +110,7 @@ public void testLocatedBlocks2Locations() { l2.setCorrupt(true); List ls = Arrays.asList(l1, l2); - LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null, 0); + LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null); BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs); 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 9cdb763f0323d..989e9fc0a5583 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 @@ -81,7 +81,7 @@ private static void clearBaseDir() throws IOException { public void testDataDirectories() throws IOException { File dataDir = new File(BASE_DIR, "data").getCanonicalFile(); Configuration conf = cluster.getConfiguration(0); - // 1. Test unsupported schema. Only "file:" is supported. + // 1. Test unsupported ecPolicy. Only "file:" is supported. String dnDir = makeURI("shv", null, fileAsURI(dataDir).getPath()); conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dnDir); DataNode dn = null; @@ -97,7 +97,7 @@ public void testDataDirectories() throws IOException { } assertNull("Data-node startup should have failed.", dn); - // 2. Test "file:" schema and no schema (path-only). Both should work. + // 2. Test "file:" ecPolicy and no ecPolicy (path-only). Both should work. String dnDir1 = fileAsURI(dataDir).toString() + "1"; String dnDir2 = makeURI("file", "localhost", fileAsURI(dataDir).getPath() + "2"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java deleted file mode 100644 index 88198c914a045..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * 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 static org.junit.Assert.*; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; -import org.apache.hadoop.io.erasurecode.ECSchema; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestECSchemas { - private MiniDFSCluster cluster; - - @Before - public void before() throws IOException { - cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0) - .build(); - cluster.waitActive(); - } - - @After - public void after() { - if (cluster != null) { - cluster.shutdown(); - } - } - - @Test - public void testGetECSchemas() throws Exception { - ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas(); - assertNotNull(ecSchemas); - assertTrue("Should have at least one schema", ecSchemas.length > 0); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java index 1f317b85998c8..6e2ec5ec95354 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java @@ -737,7 +737,7 @@ private static void mockCreate(ClientProtocol mcp, version, new byte[suite.getAlgorithmBlockSize()], new byte[suite.getAlgorithmBlockSize()], "fakeKey", "fakeVersion"), - (byte) 0, null, 0)) + (byte) 0, null)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index 1a10ebff9b8ca..a878501f5b008 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -22,10 +22,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INode; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -65,7 +65,7 @@ public void testCreateECZone() fs.mkdir(testDir, FsPermission.getDirDefault()); /* Normal creation of an erasure coding zone */ - fs.getClient().createErasureCodingZone(testDir.toString(), null, 0); + fs.getClient().createErasureCodingZone(testDir.toString(), null); /* Verify files under the zone are striped */ final Path ECFilePath = new Path(testDir, "foo"); @@ -78,7 +78,7 @@ public void testCreateECZone() fs.mkdir(notEmpty, FsPermission.getDirDefault()); fs.create(new Path(notEmpty, "foo")); try { - fs.getClient().createErasureCodingZone(notEmpty.toString(), null, 0); + fs.getClient().createErasureCodingZone(notEmpty.toString(), null); fail("Erasure coding zone on non-empty dir"); } catch (IOException e) { assertExceptionContains("erasure coding zone for a non-empty directory", e); @@ -88,10 +88,10 @@ public void testCreateECZone() final Path zone1 = new Path("/zone1"); final Path zone2 = new Path(zone1, "zone2"); fs.mkdir(zone1, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(zone1.toString(), null, 0); + fs.getClient().createErasureCodingZone(zone1.toString(), null); fs.mkdir(zone2, FsPermission.getDirDefault()); try { - fs.getClient().createErasureCodingZone(zone2.toString(), null, 0); + fs.getClient().createErasureCodingZone(zone2.toString(), null); fail("Nested erasure coding zones"); } catch (IOException e) { assertExceptionContains("already in an erasure coding zone", e); @@ -101,7 +101,7 @@ public void testCreateECZone() final Path fPath = new Path("/file"); fs.create(fPath); try { - fs.getClient().createErasureCodingZone(fPath.toString(), null, 0); + fs.getClient().createErasureCodingZone(fPath.toString(), null); fail("Erasure coding zone on file"); } catch (IOException e) { assertExceptionContains("erasure coding zone for a file", e); @@ -114,8 +114,8 @@ public void testMoveValidity() throws IOException, InterruptedException { final Path dstECDir = new Path("/dstEC"); fs.mkdir(srcECDir, FsPermission.getDirDefault()); fs.mkdir(dstECDir, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(srcECDir.toString(), null, 0); - fs.getClient().createErasureCodingZone(dstECDir.toString(), null, 0); + fs.getClient().createErasureCodingZone(srcECDir.toString(), null); + fs.getClient().createErasureCodingZone(dstECDir.toString(), null); final Path srcFile = new Path(srcECDir, "foo"); fs.create(srcFile); @@ -160,7 +160,7 @@ public void testMoveValidity() throws IOException, InterruptedException { public void testReplication() throws IOException { final Path testDir = new Path("/ec"); fs.mkdir(testDir, FsPermission.getDirDefault()); - fs.createErasureCodingZone(testDir, null, 0); + fs.createErasureCodingZone(testDir, null); final Path fooFile = new Path(testDir, "foo"); // create ec file with replication=0 fs.create(fooFile, FsPermission.getFileDefault(), true, @@ -177,47 +177,47 @@ public void testReplication() throws IOException { } @Test - public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception { + public void testGetErasureCodingInfoWithSystemDefaultECPolicy() throws Exception { String src = "/ec"; final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); // dir ECInfo before creating ec zone - assertNull(fs.getClient().getFileInfo(src).getECSchema()); + assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy()); // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, null, 0); //Default one will be used. - ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema(); - verifyErasureCodingInfo(src, sysDefaultSchema); + fs.getClient().createErasureCodingZone(src, null); //Default one will be used. + ErasureCodingPolicy sysDefaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + verifyErasureCodingInfo(src, sysDefaultECPolicy); fs.create(new Path(ecDir, "child1")).close(); // verify for the files in ec zone - verifyErasureCodingInfo(src + "/child1", sysDefaultSchema); + verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy); } @Test public void testGetErasureCodingInfo() throws Exception { - ECSchema[] sysSchemas = ErasureCodingSchemaManager.getSystemSchemas(); - assertTrue("System schemas should be of only 1 for now", - sysSchemas.length == 1); + ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices(); + assertTrue("System ecPolicies should be of only 1 for now", + sysECPolicies.length == 1); - ECSchema usingSchema = sysSchemas[0]; + ErasureCodingPolicy usingECPolicy = sysECPolicies[0]; String src = "/ec2"; final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); // dir ECInfo before creating ec zone - assertNull(fs.getClient().getFileInfo(src).getECSchema()); + assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy()); // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, usingSchema, 0); - verifyErasureCodingInfo(src, usingSchema); + fs.getClient().createErasureCodingZone(src, usingECPolicy); + verifyErasureCodingInfo(src, usingECPolicy); fs.create(new Path(ecDir, "child1")).close(); // verify for the files in ec zone - verifyErasureCodingInfo(src + "/child1", usingSchema); + verifyErasureCodingInfo(src + "/child1", usingECPolicy); } private void verifyErasureCodingInfo( - String src, ECSchema usingSchema) throws IOException { + String src, ErasureCodingPolicy usingECPolicy) throws IOException { HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src); - ECSchema schema = hdfsFileStatus.getECSchema(); - assertNotNull(schema); - assertEquals("Actually used schema should be equal with target schema", - usingSchema, schema); + ErasureCodingPolicy ecPolicy = hdfsFileStatus.getErasureCodingPolicy(); + assertNotNull(ecPolicy); + assertEquals("Actually used ecPolicy should be equal with target ecPolicy", + usingECPolicy, ecPolicy); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java similarity index 56% rename from hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java rename to hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java index 3c400b7a9563c..4610ced546040 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java @@ -9,13 +9,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.junit.After; import org.junit.Before; import org.junit.Test; -public class TestFileStatusWithECschema { +public class TestFileStatusWithECPolicy { private MiniDFSCluster cluster; private DistributedFileSystem fs; private DFSClient client; @@ -37,29 +37,29 @@ public void after() { } @Test - public void testFileStatusWithECschema() throws Exception { + public void testFileStatusWithECPolicy() throws Exception { // test directory not in EC zone final Path dir = new Path("/foo"); assertTrue(fs.mkdir(dir, FsPermission.getDirDefault())); - assertNull(client.getFileInfo(dir.toString()).getECSchema()); + assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy()); // test file not in EC zone final Path file = new Path(dir, "foo"); fs.create(file).close(); - assertNull(client.getFileInfo(file.toString()).getECSchema()); + assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy()); fs.delete(file, true); - final ECSchema schema1 = ErasureCodingSchemaManager.getSystemDefaultSchema(); + final ErasureCodingPolicy ecPolicy1 = ErasureCodingPolicyManager.getSystemDefaultPolicy(); // create EC zone on dir - fs.createErasureCodingZone(dir, schema1, 0); - final ECSchema schame2 = client.getFileInfo(dir.toUri().getPath()).getECSchema(); - assertNotNull(schame2); - assertTrue(schema1.equals(schame2)); + fs.createErasureCodingZone(dir, ecPolicy1); + final ErasureCodingPolicy ecPolicy2 = client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy(); + assertNotNull(ecPolicy2); + assertTrue(ecPolicy1.equals(ecPolicy2)); // test file in EC zone fs.create(file).close(); - final ECSchema schame3 = - fs.getClient().getFileInfo(file.toUri().getPath()).getECSchema(); - assertNotNull(schame3); - assertTrue(schema1.equals(schame3)); + final ErasureCodingPolicy ecPolicy3 = + fs.getClient().getFileInfo(file.toUri().getPath()).getErasureCodingPolicy(); + assertNotNull(ecPolicy3); + assertTrue(ecPolicy1.equals(ecPolicy3)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java index d0cd3350963f6..b77ff3a000668 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java @@ -354,12 +354,12 @@ public void testFactory() throws Exception { Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null, 0)).when(mcp).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString()); Mockito .doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0, null, 0)) + 1010, 0, null, (byte) 0, null)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 1719d3f54a1ee..cb2ec118e16eb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -64,8 +64,7 @@ public class TestReadStripedFileWithDecoding { public void setup() throws IOException { cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()) .numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", - null, cellSize); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java index 4c2438d76bcc6..38256ab256f18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java @@ -52,8 +52,7 @@ public class TestReadStripedFileWithMissingBlocks { public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", - null, cellSize); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index 9285fd73feb77..5c173592eeb1a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -78,7 +78,7 @@ public void setup() throws IOException { cluster.waitActive(); fs = cluster.getFileSystem(); - fs.getClient().createErasureCodingZone("/", null, 0); + fs.getClient().createErasureCodingZone("/", null); List datanodes = cluster.getDataNodes(); for (int i = 0; i < dnNum; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java index 6f0bc71ad5baa..f577ddba2806b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java @@ -54,8 +54,7 @@ public void setup() throws IOException { conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", - null, cellSize); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); cluster.waitActive(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 089a13438e3e4..810edb291d2be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -57,8 +57,7 @@ public class TestWriteReadStripedFile { public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", - null, cellSize); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java index 3679c5fcacdcf..deffbcc6a766e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -48,8 +48,7 @@ public class TestWriteStripedFileWithFailure { public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", - null, cellSize); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); } 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 3675e63d66596..6942ac89c4245 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 @@ -71,7 +71,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; @@ -88,7 +88,7 @@ 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.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; @@ -682,8 +682,7 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices0 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo( new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0, - liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema(), - 64 * 1024); + liveBlkIndices0, ErasureCodingPolicyManager.getSystemDefaultPolicy()); DatanodeInfo[] dnInfos1 = new DatanodeInfo[] { DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() }; DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil @@ -697,8 +696,7 @@ public void testBlockECRecoveryCommand() { short[] liveBlkIndices1 = new short[2]; BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo( new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1, - liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema(), - 64 * 1024); + liveBlkIndices1, ErasureCodingPolicyManager.getSystemDefaultPolicy()); List blkRecoveryInfosList = new ArrayList(); blkRecoveryInfosList.add(blkECRecoveryInfo0); blkRecoveryInfosList.add(blkECRecoveryInfo1); @@ -740,18 +738,18 @@ private void assertBlockECRecoveryInfoEquals( assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]); } - ECSchema ecSchema1 = blkECRecoveryInfo1.getECSchema(); - ECSchema ecSchema2 = blkECRecoveryInfo2.getECSchema(); - // Compare ECSchemas same as default ECSchema as we used system default - // ECSchema used in this test - compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema1); - compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema2); - } - - private void compareECSchemas(ECSchema ecSchema1, ECSchema ecSchema2) { - assertEquals(ecSchema1.getSchemaName(), ecSchema2.getSchemaName()); - assertEquals(ecSchema1.getNumDataUnits(), ecSchema2.getNumDataUnits()); - assertEquals(ecSchema1.getNumParityUnits(), ecSchema2.getNumParityUnits()); + ErasureCodingPolicy ecPolicy1 = blkECRecoveryInfo1.getErasureCodingPolicy(); + ErasureCodingPolicy ecPolicy2 = blkECRecoveryInfo2.getErasureCodingPolicy(); + // Compare ECPolicies same as default ECPolicy as we used system default + // ECPolicy used in this test + compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy1); + compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy2); + } + + private void compareECPolicies(ErasureCodingPolicy ecPolicy1, ErasureCodingPolicy ecPolicy2) { + assertEquals(ecPolicy1.getName(), ecPolicy2.getName()); + assertEquals(ecPolicy1.getNumDataUnits(), ecPolicy2.getNumDataUnits()); + assertEquals(ecPolicy1.getNumParityUnits(), ecPolicy2.getNumParityUnits()); } private void assertDnInfosEqual(DatanodeInfo[] dnInfos1, 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 8239e5f1d6d11..99b460a5d9546 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 @@ -1503,7 +1503,7 @@ private void doTestBalancerWithStripedFile(Configuration conf) throws Exception cluster.waitActive(); client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy(); - client.createErasureCodingZone("/", null, 0); + client.createErasureCodingZone("/", null); long totalCapacity = sum(capacities); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java index 6788770f70178..1b23600c8d23e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -19,10 +19,9 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.junit.Assert; import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; @@ -45,11 +44,10 @@ public class TestBlockInfoStriped { private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; private static final long BASE_ID = -1600; private static final Block baseBlock = new Block(BASE_ID); - private static final ECSchema testSchema - = ErasureCodingSchemaManager.getSystemDefaultSchema(); - private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); private final BlockInfoStriped info = new BlockInfoStriped(baseBlock, - testSchema, cellSize); + testECPolicy); private Block[] createReportedBlocks(int num) { Block[] blocks = new Block[num]; @@ -237,7 +235,7 @@ public void testWrite() { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutput out = new DataOutputStream(byteStream); BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes, - generationStamp), testSchema, cellSize); + generationStamp), testECPolicy); try { blk.write(out); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java index f985f54b57407..2202b342dcd8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java @@ -56,7 +56,7 @@ public void testRead() throws Exception { conf = getConf(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient() - .createErasureCodingZone("/", null, cellSize); + .createErasureCodingZone("/", null); try { cluster.waitActive(); doTestRead(conf, cluster, true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java index 2f2356f625b68..2e084fca0b1ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java @@ -86,7 +86,7 @@ public void setup() throws Exception { .getBlockGroupIdGenerator(); fs.mkdirs(eczone); cluster.getFileSystem().getClient() - .createErasureCodingZone("/eczone", null, cellSize); + .createErasureCodingZone("/eczone", null); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java index 0f419ef550fed..7cd2e19427d4a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java @@ -19,9 +19,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -31,17 +30,15 @@ public class TestUnderReplicatedBlockQueues { - private final ECSchema ecSchema = - ErasureCodingSchemaManager.getSystemDefaultSchema(); - private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final ErasureCodingPolicy ecPolicy = + ErasureCodingPolicyManager.getSystemDefaultPolicy(); private BlockInfo genBlockInfo(long id) { return new BlockInfoContiguous(new Block(id), (short) 3); } private BlockInfo genStripedBlockInfo(long id, long numBytes) { - BlockInfoStriped sblk = new BlockInfoStriped(new Block(id), ecSchema, - CELLSIZE); + BlockInfoStriped sblk = new BlockInfoStriped(new Block(id), ecPolicy); sblk.setNumBytes(numBytes); return sblk; } @@ -101,8 +98,8 @@ public void testBlockPriorities() throws Throwable { @Test public void testStripedBlockPriorities() throws Throwable { - int dataBlkNum = ecSchema.getNumDataUnits(); - int parityBlkNUm = ecSchema.getNumParityUnits(); + int dataBlkNum = ecPolicy.getNumDataUnits(); + int parityBlkNUm = ecPolicy.getNumParityUnits(); doTestStripedBlockPriorities(1, parityBlkNUm); doTestStripedBlockPriorities(dataBlkNum, parityBlkNUm); } @@ -110,7 +107,7 @@ public void testStripedBlockPriorities() throws Throwable { private void doTestStripedBlockPriorities(int dataBlkNum, int parityBlkNum) throws Throwable { int groupSize = dataBlkNum + parityBlkNum; - long numBytes = CELLSIZE * dataBlkNum; + long numBytes = ecPolicy.getCellSize() * dataBlkNum; UnderReplicatedBlocks queues = new UnderReplicatedBlocks(); // add a striped block which been left NUM_DATA_BLOCKS internal blocks diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index 29e8d24383e26..261d3975a64a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -426,7 +426,7 @@ public void testMoverWithStripedFile() throws Exception { client.setStoragePolicy(barDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME); // set "/bar" directory with EC zone. - client.createErasureCodingZone(barDir, null, 0); + client.createErasureCodingZone(barDir, null); // write file to barDir final String fooFile = "/bar/foo"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java index 337911d0cae11..7d06a9b9c68e4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java @@ -70,7 +70,7 @@ public void setup() throws IOException { cluster.waitActive(); fs = cluster.getFileSystem(); fs.mkdirs(dirPath); - fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE); + fs.getClient().createErasureCodingZone(dirPath.toString(), null); } @After @@ -180,7 +180,7 @@ public void testProcessOverReplicatedAndCorruptStripedBlock() long groupId = bg.getBlock().getBlockId(); Block blk = new Block(groupId, BLOCK_SIZE, gs); BlockInfoStriped blockInfo = new BlockInfoStriped(blk, - ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE); + ErasureCodingPolicyManager.getSystemDefaultPolicy()); for (int i = 0; i < GROUP_SIZE; i++) { blk.setBlockId(groupId + i); cluster.injectBlocks(i, Arrays.asList(blk), bpid); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index dd7086bd14075..4ae10dbe15834 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -75,7 +75,7 @@ public void setup() throws IOException { .numDataNodes(GROUP_SIZE).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); - dfs.getClient().createErasureCodingZone("/", null, 0); + dfs.getClient().createErasureCodingZone("/", null); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 5180f1336b408..aefafd028568a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -57,7 +57,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.test.PathUtils; import org.apache.log4j.Level; import org.junit.Test; @@ -76,8 +75,8 @@ public class TestFSEditLogLoader { private static final int NUM_DATA_NODES = 0; - private static final ECSchema testSchema - = ErasureCodingSchemaManager.getSystemDefaultSchema(); + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); @Test public void testDisplayRecentEditLogOpCodes() throws IOException { @@ -450,11 +449,10 @@ public void testAddNewStripedBlock() throws IOException{ long timestamp = 1426222918; short blockNum = HdfsConstants.NUM_DATA_BLOCKS; short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; - int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir, null, 0); + fs.getClient().getNamenode().createErasureCodingZone(testDir, null); // Create a file with striped block Path p = new Path(testFilePath); @@ -466,7 +464,7 @@ public void testAddNewStripedBlock() throws IOException{ // Add a striped block to the file BlockInfoStriped stripedBlk = new BlockInfoStriped( - new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize); + new Block(blkId, blkNumBytes, timestamp), testECPolicy); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); file.addBlock(stripedBlk); @@ -491,7 +489,6 @@ public void testAddNewStripedBlock() throws IOException{ assertEquals(timestamp, blks[0].getGenerationStamp()); assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum()); assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum()); - assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize()); cluster.shutdown(); cluster = null; @@ -524,17 +521,16 @@ public void testUpdateStripedBlocks() throws IOException{ long timestamp = 1426222918; short blockNum = HdfsConstants.NUM_DATA_BLOCKS; short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; - int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir, null, 0); + fs.getClient().getNamenode().createErasureCodingZone(testDir, null); //create a file with striped blocks Path p = new Path(testFilePath); DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); BlockInfoStriped stripedBlk = new BlockInfoStriped( - new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize); + new Block(blkId, blkNumBytes, timestamp), testECPolicy); INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); file.toUnderConstruction(clientName, clientMachine); file.addBlock(stripedBlk); @@ -573,7 +569,6 @@ public void testUpdateStripedBlocks() throws IOException{ assertEquals(newTimestamp, blks[0].getGenerationStamp()); assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum()); assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum()); - assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize()); cluster.shutdown(); cluster = null; 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 a14492294fb4d..d1c760059f4ba 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 @@ -28,11 +28,11 @@ import java.util.EnumSet; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.Assert; import org.apache.hadoop.fs.permission.PermissionStatus; @@ -68,9 +68,8 @@ public class TestFSImage { private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ = "image-with-zero-block-size.tar.gz"; - private static final ECSchema testSchema - = ErasureCodingSchemaManager.getSystemDefaultSchema(); - private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); @Test public void testPersist() throws IOException { @@ -141,7 +140,7 @@ private void testPersistHelper(Configuration conf) throws IOException { private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf, boolean isUC) throws IOException{ // contruct a INode with StripedBlock for saving and loading - fsn.createErasureCodingZone("/", null, 0, false); + fsn.createErasureCodingZone("/", null, false); long id = 123456789; byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes(); PermissionStatus permissionStatus = new PermissionStatus("testuser_a", @@ -162,7 +161,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con for (int i = 0; i < stripedBlks.length; i++) { stripedBlks[i] = new BlockInfoStriped( new Block(stripedBlkId + i, preferredBlockSize, timestamp), - testSchema, cellSize); + testECPolicy); file.addBlock(stripedBlks[i]); } @@ -386,7 +385,7 @@ public void testSupportBlockGroup() throws IOException { .build(); cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); - fs.getClient().getNamenode().createErasureCodingZone("/", null, 0); + fs.getClient().getNamenode().createErasureCodingZone("/", null); Path file = new Path("/striped"); FSDataOutputStream out = fs.create(file); byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE); 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 7e2b41d4dab0a..e6f505ef4cfab 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 @@ -1202,7 +1202,7 @@ public void testFsckFileNotFound() throws Exception { HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication, blockSize, modTime, accessTime, perms, owner, group, symlink, - path, fileId, numChildren, null, storagePolicy, null, 0); + path, fileId, numChildren, null, storagePolicy, null); Result replRes = new ReplicationResult(conf); Result ecRes = new ErasureCodingResult(conf); @@ -1644,8 +1644,8 @@ public void testECFsck() throws Exception { final long precision = 1L; conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision); conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L); - int totalSize = ErasureCodingSchemaManager.getSystemDefaultSchema().getNumDataUnits() - + ErasureCodingSchemaManager.getSystemDefaultSchema().getNumParityUnits(); + int totalSize = ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumDataUnits() + + ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumParityUnits(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build(); fs = cluster.getFileSystem(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java index 9c585a4a2380c..b1bb191c37b93 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java @@ -26,11 +26,11 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -44,10 +44,10 @@ public class TestQuotaWithStripedBlocks { private static final int BLOCK_SIZE = 1024 * 1024; private static final long DISK_QUOTA = BLOCK_SIZE * 10; - private static final ECSchema ecSchema = - ErasureCodingSchemaManager.getSystemDefaultSchema(); - private static final int NUM_DATA_BLOCKS = ecSchema.getNumDataUnits(); - private static final int NUM_PARITY_BLOCKS = ecSchema.getNumParityUnits(); + private static final ErasureCodingPolicy ecPolicy = + ErasureCodingPolicyManager.getSystemDefaultPolicy(); + private static final int NUM_DATA_BLOCKS = ecPolicy.getNumDataUnits(); + private static final int NUM_PARITY_BLOCKS = ecPolicy.getNumParityUnits(); private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; private static final Path ecDir = new Path("/ec"); @@ -66,7 +66,7 @@ public void setUp() throws IOException { dfs = cluster.getFileSystem(); dfs.mkdirs(ecDir); - dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema, 0); + dfs.getClient().createErasureCodingZone(ecDir.toString(), ecPolicy); dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA); dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA); dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index a9b2048416837..7684c1d91948c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -35,13 +35,13 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; -import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.Test; @@ -59,9 +59,8 @@ public class TestStripedINodeFile { private final BlockStoragePolicy defaultPolicy = defaultSuite.getDefaultPolicy(); - private static final ECSchema testSchema - = ErasureCodingSchemaManager.getSystemDefaultSchema(); - private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); private static INodeFile createStripedINodeFile() { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, @@ -79,7 +78,7 @@ public void testBlockStripedFeature() public void testBlockStripedTotalBlockCount() { Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema, cellSize); + = new BlockInfoStriped(blk, testECPolicy); assertEquals(9, blockInfoStriped.getTotalBlockNum()); } @@ -89,7 +88,7 @@ public void testBlockStripedLength() INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema, cellSize); + = new BlockInfoStriped(blk, testECPolicy); inf.addBlock(blockInfoStriped); assertEquals(1, inf.getBlocks().length); } @@ -100,7 +99,7 @@ public void testBlockStripedConsumedSpace() INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema, cellSize); + = new BlockInfoStriped(blk, testECPolicy); blockInfoStriped.setNumBytes(1); inf.addBlock(blockInfoStriped); // 0. Calculate the total bytes per stripes @@ -125,11 +124,11 @@ public void testMultipleBlockStripedConsumedSpace() INodeFile inf = createStripedINodeFile(); Block blk1 = new Block(1); BlockInfoStriped blockInfoStriped1 - = new BlockInfoStriped(blk1, testSchema, cellSize); + = new BlockInfoStriped(blk1, testECPolicy); blockInfoStriped1.setNumBytes(1); Block blk2 = new Block(2); BlockInfoStriped blockInfoStriped2 - = new BlockInfoStriped(blk2, testSchema, cellSize); + = new BlockInfoStriped(blk2, testECPolicy); blockInfoStriped2.setNumBytes(1); inf.addBlock(blockInfoStriped1); inf.addBlock(blockInfoStriped2); @@ -144,7 +143,7 @@ public void testBlockStripedFileSize() INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema, cellSize); + = new BlockInfoStriped(blk, testECPolicy); blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); // Compute file size should return actual data @@ -159,7 +158,7 @@ public void testBlockUCStripedFileSize() INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); BlockInfoUnderConstructionStriped bInfoUCStriped - = new BlockInfoUnderConstructionStriped(blk, testSchema, cellSize); + = new BlockInfoUnderConstructionStriped(blk, testECPolicy); bInfoUCStriped.setNumBytes(100); inf.addBlock(bInfoUCStriped); assertEquals(100, inf.computeFileSize()); @@ -172,7 +171,7 @@ public void testBlockStripedComputeQuotaUsage() INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); BlockInfoStriped blockInfoStriped - = new BlockInfoStriped(blk, testSchema, cellSize); + = new BlockInfoStriped(blk, testECPolicy); blockInfoStriped.setNumBytes(100); inf.addBlock(blockInfoStriped); @@ -193,7 +192,7 @@ public void testBlockUCStripedComputeQuotaUsage() INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); BlockInfoUnderConstructionStriped bInfoUCStriped - = new BlockInfoUnderConstructionStriped(blk, testSchema, cellSize); + = new BlockInfoUnderConstructionStriped(blk, testECPolicy); bInfoUCStriped.setNumBytes(100); inf.addBlock(bInfoUCStriped); @@ -235,7 +234,7 @@ public void testDeleteOp() throws Exception { dfs.mkdirs(zone); // create erasure zone - dfs.createErasureCodingZone(zone, null, 0); + dfs.createErasureCodingZone(zone, null); DFSTestUtil.createFile(dfs, zoneFile, len, (short) 1, 0xFEED); DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED); final FSDirectory fsd = fsn.getFSDirectory(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java index f4efbcf0d1f82..7bfaab67228d0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java @@ -60,7 +60,7 @@ public static void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.waitActive(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0); + cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); Path eczone = new Path("/eczone"); fs.mkdirs(eczone); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java index 5d85073994f3b..e61ac07e975a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -29,8 +29,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.*; -import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager; -import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.junit.Before; import org.junit.Test; @@ -84,8 +84,8 @@ public class TestStripedBlockUtil { private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE; /** number of full stripes in a full block group */ private final int BLK_GROUP_STRIPE_NUM = 16; - private final ECSchema SCEHMA = ErasureCodingSchemaManager. - getSystemDefaultSchema(); + private final ErasureCodingPolicy ECPOLICY = ErasureCodingPolicyManager. + getSystemDefaultPolicy(); private final Random random = new Random(); private int[] blockGroupSizes; @@ -152,7 +152,7 @@ private byte[][] createInternalBlkBuffers(int bgSize) { int done = 0; while (done < bgSize) { Preconditions.checkState(done % CELLSIZE == 0); - StripingCell cell = new StripingCell(SCEHMA, CELLSIZE, done / CELLSIZE, 0); + StripingCell cell = new StripingCell(ECPOLICY, CELLSIZE, done / CELLSIZE, 0); int idxInStripe = cell.idxInStripe; int size = Math.min(CELLSIZE, bgSize - done); for (int i = 0; i < size; i++) { @@ -245,7 +245,7 @@ public void testDivideByteRangeIntoStripes() { if (brStart + brSize > bgSize) { continue; } - AlignedStripe[] stripes = divideByteRangeIntoStripes(SCEHMA, + AlignedStripe[] stripes = divideByteRangeIntoStripes(ECPOLICY, CELLSIZE, blockGroup, brStart, brStart + brSize - 1, assembled, 0); for (AlignedStripe stripe : stripes) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java index 303d063ecc65a..8947c5b49a16d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java @@ -65,7 +65,7 @@ public void testHdfsFileStatus() throws IOException { final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26, now, now + 10, new FsPermission((short) 0644), "user", "group", DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"), - HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null, 0); + HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null); final FileStatus fstatus = toFileStatus(status, parent); System.out.println("status = " + status); System.out.println("fstatus = " + fstatus); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml index 70020d54db494..5e6065843f89c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -57,11 +57,11 @@ RegexpComparator - ^[ \t]*Create a zone to encode files using a specified schema( )* + ^[ \t]*Create a zone to encode files using a specified policy( )* RegexpComparator - ^-createZone \[-s <schemaName>\] \[-c <cellSize>\] <path>(.)* + ^-createZone \[-s <policyName>\] <path>(.)* @@ -86,20 +86,20 @@ - help: listSchemas command + help: listPolicies command - -fs NAMENODE -help listSchemas + -fs NAMENODE -help listPolicies SubstringComparator - Get the list of ECSchemas supported + Get the list of erasure coding policies supported RegexpComparator - ^-listSchemas (.)* + ^-listPolicies (.)* @@ -109,7 +109,7 @@ createZone : create a zone to encode files -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s RS-6-3 /eczone + -fs NAMENODE -createZone -s RS-6-3-64k /eczone -fs NAMENODE -rmdir /eczone @@ -141,7 +141,7 @@ - createZone : default schema + createZone : default policy -fs NAMENODE -mkdir /eczone -fs NAMENODE -createZone /eczone @@ -153,7 +153,7 @@ SubstringComparator - Dir: /eczone, Schema: ECSchema=[Name=RS-6-3 + Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k @@ -179,7 +179,7 @@ getZone : get information about the EC zone at specified path -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s RS-6-3 /eczone + -fs NAMENODE -createZone -s RS-6-3-64k /eczone -fs NAMENODE -getZone /eczone @@ -188,7 +188,7 @@ SubstringComparator - Dir: /eczone, Schema: ECSchema=[Name=RS-6-3 + Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k @@ -197,7 +197,7 @@ getZone : get EC zone at specified file path -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s RS-6-3 /eczone + -fs NAMENODE -createZone -s RS-6-3-64k /eczone -fs NAMENODE -touchz /eczone/ecfile -fs NAMENODE -getZone /eczone/ecfile @@ -208,15 +208,15 @@ SubstringComparator - Dir: /eczone, Schema: ECSchema=[Name=RS-6-3 + Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k - listSchemas : get the list of ECSchemas supported + listPolicies : get the list of ECPolicies supported - -fs NAMENODE -listSchemas + -fs NAMENODE -listPolicies @@ -247,7 +247,7 @@ - createZone : illegal parameters - schema name is missing + createZone : illegal parameters - policy name is missing -fs NAMENODE -mkdir /eczone -fs NAMENODE -createZone -s @@ -281,10 +281,10 @@ - createZone : illegal parameters - invalidschema + createZone : illegal parameters - invalidpolicy -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s invalidschema /eczone + -fs NAMENODE -createZone -s invalidpolicy /eczone -fs NAMENODE -rmdir /eczone @@ -292,7 +292,7 @@ SubstringComparator - Schema 'invalidschema' does not match any of the supported schemas. Please select any one of [RS-6-3] + Policy 'invalidpolicy' does not match any of the supported policies. Please select any one of [RS-6-3-64k] @@ -359,16 +359,16 @@ - listSchemas : illegal parameters - too many parameters + listPolicies : illegal parameters - too many parameters - -fs NAMENODE -listSchemas /eczone + -fs NAMENODE -listPolicies /eczone SubstringComparator - -listSchemas: Too many parameters + -listPolicies: Too many parameters From b57c9a35f76612a02e16497ea92fbc3f4dc524ec Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 14 Aug 2015 15:16:22 -0700 Subject: [PATCH 194/212] HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the blocklocations which doesn't satisfy BlockGroupSize. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/StripedDataStreamer.java | 42 ++++++++-- ...TestDFSStripedOutputStreamWithFailure.java | 84 +++++++++++++++++++ 3 files changed, 120 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index b752d5bdeb741..dad997a06b947 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -397,3 +397,6 @@ HDFS-8854. Erasure coding: add ECPolicy to replace schema+cellSize in hadoop-hdfs. (Walter Su via zhz) + + HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the + blocklocations which doesn't satisfy BlockGroupSize. (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 2d51dc43588ef..f533bf9b10964 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue; 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.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; @@ -167,18 +168,33 @@ void populate() throws IOException { final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock( excludedNodes); + if (lb.getLocations().length < HdfsConstants.NUM_DATA_BLOCKS) { + throw new IOException( + "Failed to get datablocks number of nodes from namenode: blockGroupSize= " + + (HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS) + + ", blocks.length= " + lb.getLocations().length); + } final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( (LocatedStripedBlock)lb, BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); for (int i = 0; i < blocks.length; i++) { - if (!coordinator.getStripedDataStreamer(i).isFailed()) { - if (blocks[i] == null) { - getLastException().set( - new IOException("Failed to get following block, i=" + i)); - } else { - followingBlocks.offer(i, blocks[i]); - } + StripedDataStreamer si = coordinator.getStripedDataStreamer(i); + if (si.isFailed()) { + continue; // skipping failed data streamer + } + if (blocks[i] == null) { + // Set exception and close streamer as there is no block locations + // found for the parity block. + LOG.warn("Failed to get block location for parity block, index=" + + i); + si.getLastException().set( + new IOException("Failed to get following block, i=" + i)); + si.setFailed(true); + si.endBlock(); + si.close(true); + } else { + followingBlocks.offer(i, blocks[i]); } } } @@ -199,7 +215,11 @@ void populate() throws IOException { .parseStripedBlockGroup((LocatedStripedBlock) updated, BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { - final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock(); + StripedDataStreamer si = coordinator.getStripedDataStreamer(i); + if (si.isFailed()) { + continue; // skipping failed data streamer + } + final ExtendedBlock bi = si.getBlock(); if (bi != null) { final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS), null, null, null, -1, updated.isCorrupt(), null); @@ -225,7 +245,11 @@ void populate() throws IOException { final ExtendedBlock newBG = newBlock(bg, newGS); final ExtendedBlock updated = callUpdatePipeline(bg, newBG); for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { - final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock(); + StripedDataStreamer si = coordinator.getStripedDataStreamer(i); + if (si.isFailed()) { + continue; // skipping failed data streamer + } + final ExtendedBlock bi = si.getBlock(); updateBlocks.offer(i, newBlock(bi, updated.getGenerationStamp())); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index fed9f167643f0..f65d0c750003c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hdfs; +import static org.junit.Assert.assertEquals; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -33,6 +35,7 @@ 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.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; @@ -40,6 +43,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.security.token.Token; @@ -145,6 +149,86 @@ public void testBlockTokenExpired() throws Exception { } } + @Test(timeout = 90000) + public void testAddBlockWhenNoSufficientDataBlockNumOfNodes() + throws IOException { + HdfsConfiguration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + try { + setup(conf); + ArrayList dataNodes = cluster.getDataNodes(); + // shutdown few datanodes to avoid getting sufficient data blocks number + // of datanodes + int killDns = dataNodes.size() / 2; + int numDatanodes = dataNodes.size() - killDns; + for (int i = 0; i < killDns; i++) { + cluster.stopDataNode(i); + } + cluster.restartNameNodes(); + cluster.triggerHeartbeats(); + DatanodeInfo[] info = dfs.getClient().datanodeReport( + DatanodeReportType.LIVE); + assertEquals("Mismatches number of live Dns ", numDatanodes, info.length); + final Path dirFile = new Path(dir, "ecfile"); + FSDataOutputStream out = null; + try { + out = dfs.create(dirFile, true); + out.write("something".getBytes()); + out.flush(); + out.close(); + Assert.fail("Failed to validate available dns against blkGroupSize"); + } catch (IOException ioe) { + // expected + GenericTestUtils.assertExceptionContains("Failed: the number of " + + "remaining blocks = 5 < the number of data blocks = 6", ioe); + DFSStripedOutputStream dfsout = (DFSStripedOutputStream) out + .getWrappedStream(); + + // get leading streamer and verify the last exception + StripedDataStreamer datastreamer = dfsout.getStripedDataStreamer(0); + try { + datastreamer.getLastException().check(true); + Assert.fail("Failed to validate available dns against blkGroupSize"); + } catch (IOException le) { + GenericTestUtils.assertExceptionContains( + "Failed to get datablocks number of nodes from" + + " namenode: blockGroupSize= 9, blocks.length= " + + numDatanodes, le); + } + } + } finally { + tearDown(); + } + } + + @Test(timeout = 90000) + public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException { + HdfsConfiguration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + try { + setup(conf); + ArrayList dataNodes = cluster.getDataNodes(); + // shutdown few data nodes to avoid writing parity blocks + int killDns = (NUM_PARITY_BLOCKS - 1); + int numDatanodes = dataNodes.size() - killDns; + for (int i = 0; i < killDns; i++) { + cluster.stopDataNode(i); + } + cluster.restartNameNodes(); + cluster.triggerHeartbeats(); + DatanodeInfo[] info = dfs.getClient().datanodeReport( + DatanodeReportType.LIVE); + assertEquals("Mismatches number of live Dns ", numDatanodes, info.length); + Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes"); + int fileLength = HdfsConstants.BLOCK_STRIPED_CELL_SIZE - 1000; + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(dfs, srcPath, new String(expected)); + StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength); + } finally { + tearDown(); + } + } + private void runTest(final Path p, final int length, final int killPos, final int dnIndex, final boolean tokenExpire) throws Exception { LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos From 067ec8c2b14fb0929dc348b763383838e06ff8a5 Mon Sep 17 00:00:00 2001 From: Walter Su Date: Thu, 27 Aug 2015 09:09:52 +0800 Subject: [PATCH 195/212] HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream when the data length is small. Contributed by Tsz Wo Nicholas Sze. --- .../hadoop/util/ShutdownHookManager.java | 6 + .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSOutputStream.java | 6 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 56 +++-- .../org/apache/hadoop/hdfs/DataStreamer.java | 43 ++-- .../hadoop/hdfs/StripedDataStreamer.java | 138 ++++++++++--- .../server/namenode/FSDirWriteFileOp.java | 3 + .../apache/hadoop/hdfs/MiniDFSCluster.java | 2 + .../hdfs/TestDFSStripedOutputStream.java | 2 +- ...TestDFSStripedOutputStreamWithFailure.java | 195 ++++++++++++++---- ...tDFSStripedOutputStreamWithFailure000.java | 22 ++ ...tDFSStripedOutputStreamWithFailure010.java | 22 ++ 12 files changed, 384 insertions(+), 114 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java index 989c96a8e3789..85533dbffaedc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java @@ -187,4 +187,10 @@ public boolean isShutdownInProgress() { return shutdownInProgress.get(); } + /** + * clear all registered shutdownHooks. + */ + public void clearShutdownHooks() { + hooks.clear(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index dad997a06b947..8b25e68c04b92 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -400,3 +400,6 @@ HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the blocklocations which doesn't satisfy BlockGroupSize. (Rakesh R via zhz) + + HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream + when the data length is small. (szetszwo via waltersu4549) 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 00f3a6505cc7a..1654a26ce230c 100755 --- 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 @@ -406,13 +406,13 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, if (currentPacket == null) { currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer() .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false); - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + + if (LOG.isDebugEnabled()) { + LOG.debug("WriteChunk allocating new packet seqno=" + currentPacket.getSeqno() + ", src=" + src + ", packetSize=" + packetSize + ", chunksPerPacket=" + chunksPerPacket + - ", bytesCurBlock=" + getStreamer().getBytesCurBlock()); + ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index 4ca8fe6b42321..d3a054affeea0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -170,15 +170,18 @@ ExtendedBlock getBlockGroup() { } final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0; + final ExtendedBlock block = new ExtendedBlock(b0); - long numBytes = b0.getNumBytes(); - for (int i = 1; i < numDataBlocks; i++) { + long numBytes = atBlockGroupBoundary? b0.getNumBytes(): s0.getBytesCurBlock(); + for (int i = 1; i < numAllBlocks; i++) { final StripedDataStreamer si = getStripedDataStreamer(i); final ExtendedBlock bi = si.getBlock(); if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) { block.setGenerationStamp(bi.getGenerationStamp()); } - numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock(); + if (i < numDataBlocks) { + numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock(); + } } block.setNumBytes(numBytes); if (LOG.isDebugEnabled()) { @@ -318,8 +321,7 @@ private synchronized StripedDataStreamer getCurrentStreamer() { return (StripedDataStreamer)streamer; } - private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) - throws IOException { + private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) { // backup currentPacket for current streamer int oldIdx = streamers.indexOf(streamer); if (oldIdx >= 0) { @@ -349,11 +351,11 @@ private static void encode(RawErasureEncoder encoder, int numData, } - private void checkStreamers() throws IOException { + private void checkStreamers(boolean setExternalError) throws IOException { int count = 0; for(StripedDataStreamer s : streamers) { if (!s.isFailed()) { - if (s.getBlock() != null) { + if (setExternalError && s.getBlock() != null) { s.getErrorState().initExternalError(); } count++; @@ -369,11 +371,16 @@ private void checkStreamers() throws IOException { } } - private void handleStreamerFailure(String err, - Exception e) throws IOException { + private void handleStreamerFailure(String err, Exception e) + throws IOException { + handleStreamerFailure(err, e, true); + } + + private void handleStreamerFailure(String err, Exception e, + boolean setExternalError) throws IOException { LOG.warn("Failed: " + err + ", " + this, e); getCurrentStreamer().setFailed(true); - checkStreamers(); + checkStreamers(setExternalError); currentPacket = null; } @@ -505,10 +512,10 @@ private long getCurrentSumBytes() { return sum; } - private void writeParityCellsForLastStripe() throws IOException { + private boolean generateParityCellsForLastStripe() { final long currentBlockGroupBytes = getCurrentSumBytes(); if (currentBlockGroupBytes % stripeDataSize() == 0) { - return; + return false; } final int firstCellSize = @@ -530,8 +537,7 @@ private void writeParityCellsForLastStripe() throws IOException { } buffers[i].flip(); } - - writeParityCells(); + return true; } void writeParityCells() throws IOException { @@ -603,12 +609,14 @@ protected synchronized void closeImpl() throws IOException { // flush from all upper layers try { flushBuffer(); - // if the last stripe is incomplete, generate and write parity cells - writeParityCellsForLastStripe(); - enqueueAllCurrentPackets(); } catch(Exception e) { - handleStreamerFailure("closeImpl", e); + handleStreamerFailure("flushBuffer " + getCurrentStreamer(), e); + } + // if the last stripe is incomplete, generate and write parity cells + if (generateParityCellsForLastStripe()) { + writeParityCells(); } + enqueueAllCurrentPackets(); for (int i = 0; i < numAllBlocks; i++) { final StripedDataStreamer s = setCurrentStreamer(i); @@ -620,7 +628,7 @@ protected synchronized void closeImpl() throws IOException { // flush all data to Datanode flushInternal(); } catch(Exception e) { - handleStreamerFailure("closeImpl", e); + handleStreamerFailure("flushInternal " + s, e, false); } } } @@ -643,9 +651,13 @@ protected synchronized void closeImpl() throws IOException { private void enqueueAllCurrentPackets() throws IOException { int idx = streamers.indexOf(getCurrentStreamer()); for(int i = 0; i < streamers.size(); i++) { - setCurrentStreamer(i); - if (currentPacket != null) { - enqueueCurrentPacket(); + final StripedDataStreamer si = setCurrentStreamer(i); + if (!si.isFailed() && currentPacket != null) { + try { + enqueueCurrentPacket(); + } catch (IOException e) { + handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e, false); + } } } setCurrentStreamer(idx); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index c78199ed3067d..bbcdd1fb833c3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -173,7 +173,7 @@ private static void releaseBuffer(List packets, ByteArrayManager bam) packets.clear(); } - static class LastExceptionInStreamer { + class LastExceptionInStreamer { private IOException thrown; synchronized void set(Throwable t) { @@ -191,7 +191,8 @@ synchronized void check(boolean resetToNull) throws IOException { if (thrown != null) { if (LOG.isTraceEnabled()) { // wrap and print the exception to know when the check is called - LOG.trace("Got Exception while checking", new Throwable(thrown)); + LOG.trace("Got Exception while checking, " + DataStreamer.this, + new Throwable(thrown)); } final IOException e = thrown; if (resetToNull) { @@ -584,16 +585,13 @@ public void run() { } // get new block from namenode. + if (LOG.isDebugEnabled()) { + LOG.debug("stage=" + stage + ", " + this); + } if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) { - if(LOG.isDebugEnabled()) { - LOG.debug("Allocating new block " + this); - } setPipeline(nextBlockOutputStream()); initDataStreaming(); } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) { - if(LOG.isDebugEnabled()) { - LOG.debug("Append to block " + block); - } setupPipelineForAppendOrRecovery(); if (streamerClosed) { continue; @@ -639,8 +637,7 @@ public void run() { } if (LOG.isDebugEnabled()) { - LOG.debug("DataStreamer block " + block + - " sending packet " + one); + LOG.debug(this + " sending " + one); } // write out data to remote datanode @@ -1426,16 +1423,21 @@ static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) { /** update pipeline at the namenode */ ExtendedBlock updatePipeline(long newGS) throws IOException { final ExtendedBlock newBlock = newBlock(block, newGS); - return callUpdatePipeline(block, newBlock); + return callUpdatePipeline(block, newBlock, nodes, storageIDs); } - ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock) + ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock, + DatanodeInfo[] newNodes, String[] newStorageIDs) throws IOException { dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock, newBlock, - nodes, storageIDs); + newNodes, newStorageIDs); return newBlock; } + int getNumBlockWriteRetry() { + return dfsClient.getConf().getNumBlockWriteRetry(); + } + /** * Open a DataStreamer to a DataNode so that it can be written to. * This happens when a file is created and each time a new block is allocated. @@ -1446,7 +1448,7 @@ private LocatedBlock nextBlockOutputStream() throws IOException { LocatedBlock lb = null; DatanodeInfo[] nodes = null; StorageType[] storageTypes = null; - int count = dfsClient.getConf().getNumBlockWriteRetry(); + int count = getNumBlockWriteRetry(); boolean success = false; ExtendedBlock oldBlock = block; do { @@ -1502,7 +1504,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, String firstBadLink = ""; boolean checkRestart = false; if (LOG.isDebugEnabled()) { - LOG.debug("pipeline = " + Arrays.asList(nodes)); + LOG.debug("pipeline = " + Arrays.toString(nodes) + ", " + this); } // persist blocks on namenode on next flush @@ -1574,7 +1576,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, errorState.reset(); } catch (IOException ie) { if (!errorState.isRestartingNode()) { - LOG.info("Exception in createBlockOutputStream", ie); + LOG.info("Exception in createBlockOutputStream " + this, ie); } if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { LOG.info("Will fetch a new encryption key and retry, " @@ -1649,7 +1651,7 @@ private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) { } } - protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) + LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) throws IOException { final DfsClientConf conf = dfsClient.getConf(); int retries = conf.getNumBlockWriteLocateFollowingRetry(); @@ -1755,6 +1757,10 @@ DatanodeInfo[] getNodes() { return nodes; } + String[] getStorageIDs() { + return storageIDs; + } + /** * return the token of the block * @@ -1933,7 +1939,6 @@ void closeSocket() throws IOException { @Override public String toString() { - return (block == null? null: block.getLocalBlock()) - + "@" + Arrays.toString(getNodes()); + return block == null? "block==null": "" + block.getLocalBlock(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index f533bf9b10964..a20caa53326f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; import java.io.IOException; +import java.io.InterruptedIOException; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator; @@ -39,6 +40,8 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; +import com.google.common.annotations.VisibleForTesting; + /** * This class extends {@link DataStreamer} to support writing striped blocks * to datanodes. @@ -58,13 +61,13 @@ public class StripedDataStreamer extends DataStreamer { * @param the queue entry type. */ static abstract class ConcurrentPoll { - private final MultipleBlockingQueue queue; + final MultipleBlockingQueue queue; ConcurrentPoll(MultipleBlockingQueue queue) { this.queue = queue; } - T poll(final int i) throws IOException { + T poll(final int i) throws InterruptedIOException { for(;;) { synchronized(queue) { final T polled = queue.poll(i); @@ -72,18 +75,17 @@ T poll(final int i) throws IOException { return polled; } if (isReady2Populate()) { - populate(); - return queue.poll(i); + try { + populate(); + return queue.poll(i); + } catch(IOException ioe) { + LOG.warn("Failed to populate, " + this, ioe); + } } } // sleep and then retry. - try { - Thread.sleep(100); - } catch(InterruptedException ie) { - throw DFSUtil.toInterruptedIOException( - "Sleep interrupted during poll", ie); - } + sleep(100, "poll"); } } @@ -94,6 +96,15 @@ boolean isReady2Populate() { abstract void populate() throws IOException; } + private static void sleep(long ms, String op) throws InterruptedIOException { + try { + Thread.sleep(ms); + } catch(InterruptedException ie) { + throw DFSUtil.toInterruptedIOException( + "Sleep interrupted during " + op, ie); + } + } + private final Coordinator coordinator; private final int index; private volatile boolean failed; @@ -135,11 +146,14 @@ protected void endBlock() { } @Override - protected LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes) + int getNumBlockWriteRetry() { + return 0; + } + + @Override + LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes) throws IOException { - final MultipleBlockingQueue followingBlocks - = coordinator.getFollowingBlocks(); - return new ConcurrentPoll(followingBlocks) { + return new ConcurrentPoll(coordinator.getFollowingBlocks()) { @Override boolean isReady2Populate() { return super.isReady2Populate() @@ -194,18 +208,24 @@ void populate() throws IOException { si.endBlock(); si.close(true); } else { - followingBlocks.offer(i, blocks[i]); + queue.offer(i, blocks[i]); } } } }.poll(index); } + @VisibleForTesting + LocatedBlock peekFollowingBlock() { + return coordinator.getFollowingBlocks().peek(index); + } + @Override LocatedBlock updateBlockForPipeline() throws IOException { - final MultipleBlockingQueue newBlocks - = coordinator.getNewBlocks(); - return new ConcurrentPoll(newBlocks) { + if (LOG.isDebugEnabled()) { + LOG.debug("updateBlockForPipeline(), " + this); + } + return new ConcurrentPoll(coordinator.getNewBlocks()) { @Override void populate() throws IOException { final ExtendedBlock bg = coordinator.getBlockGroup(); @@ -224,10 +244,22 @@ void populate() throws IOException { final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS), null, null, null, -1, updated.isCorrupt(), null); lb.setBlockToken(updatedBlks[i].getBlockToken()); - newBlocks.offer(i, lb); + queue.offer(i, lb); } else { - final LocatedBlock lb = coordinator.getFollowingBlocks().peek(i); - lb.getBlock().setGenerationStamp(newGS); + final MultipleBlockingQueue followingBlocks + = coordinator.getFollowingBlocks(); + synchronized(followingBlocks) { + final LocatedBlock lb = followingBlocks.peek(i); + if (lb != null) { + lb.getBlock().setGenerationStamp(newGS); + si.getErrorState().reset(); + continue; + } + } + + //streamer i just have polled the block, sleep and retry. + sleep(100, "updateBlockForPipeline, " + this); + i--; } } } @@ -236,21 +268,64 @@ void populate() throws IOException { @Override ExtendedBlock updatePipeline(final long newGS) throws IOException { - final MultipleBlockingQueue updateBlocks - = coordinator.getUpdateBlocks(); - return new ConcurrentPoll(updateBlocks) { + if (LOG.isDebugEnabled()) { + LOG.debug("updatePipeline(newGS=" + newGS + "), " + this); + } + return new ConcurrentPoll(coordinator.getUpdateBlocks()) { @Override void populate() throws IOException { + final MultipleBlockingQueue followingBlocks + = coordinator.getFollowingBlocks(); final ExtendedBlock bg = coordinator.getBlockGroup(); final ExtendedBlock newBG = newBlock(bg, newGS); - final ExtendedBlock updated = callUpdatePipeline(bg, newBG); - for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { - StripedDataStreamer si = coordinator.getStripedDataStreamer(i); - if (si.isFailed()) { - continue; // skipping failed data streamer + + final int n = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + final DatanodeInfo[] newNodes = new DatanodeInfo[n]; + final String[] newStorageIDs = new String[n]; + for (int i = 0; i < n; i++) { + final StripedDataStreamer si = coordinator.getStripedDataStreamer(i); + DatanodeInfo[] nodes = si.getNodes(); + String[] storageIDs = si.getStorageIDs(); + if (nodes == null || storageIDs == null) { + synchronized(followingBlocks) { + final LocatedBlock lb = followingBlocks.peek(i); + if (lb != null) { + nodes = lb.getLocations(); + storageIDs = lb.getStorageIDs(); + } + } + } + if (nodes != null && storageIDs != null) { + newNodes[i] = nodes[0]; + newStorageIDs[i] = storageIDs[0]; + } else { + //streamer i just have polled the block, sleep and retry. + sleep(100, "updatePipeline, " + this); + i--; } + } + final ExtendedBlock updated = callUpdatePipeline(bg, newBG, newNodes, + newStorageIDs); + + for (int i = 0; i < n; i++) { + final StripedDataStreamer si = coordinator.getStripedDataStreamer(i); final ExtendedBlock bi = si.getBlock(); - updateBlocks.offer(i, newBlock(bi, updated.getGenerationStamp())); + if (bi != null) { + queue.offer(i, newBlock(bi, updated.getGenerationStamp())); + } else if (!si.isFailed()) { + synchronized(followingBlocks) { + final LocatedBlock lb = followingBlocks.peek(i); + if (lb != null) { + lb.getBlock().setGenerationStamp(newGS); + si.getErrorState().reset(); + continue; + } + } + + //streamer i just have polled the block, sleep and retry. + sleep(100, "updatePipeline, " + this); + i--; + } } } }.poll(index); @@ -258,7 +333,6 @@ void populate() throws IOException { @Override public String toString() { - return "#" + index + ": failed? " + Boolean.toString(failed).charAt(0) - + ", " + super.toString(); + return "#" + index + ": " + (failed? "failed, ": "") + super.toString(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index b9466f694050f..ffd8fbcddfdf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -134,6 +134,9 @@ static void abandonBlock( FSNamesystem fsn = fsd.getFSNamesystem(); final INodeFile file = fsn.checkLease(src, holder, inode, fileId); Preconditions.checkState(file.isUnderConstruction()); + if (file.isStriped()) { + return; // do not abandon block for striped file + } Block localBlock = ExtendedBlock.getLocalBlock(b); fsd.writeLock(); 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 65e26df971f2b..59daba4ee04b3 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 @@ -120,6 +120,7 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.ToolRunner; import com.google.common.base.Joiner; @@ -1867,6 +1868,7 @@ public void shutdown(boolean deleteDfsDir, boolean closeFileSystem) { nameNode = null; } } + ShutdownHookManager.get().clearShutdownHooks(); if (base_dir != null) { if (deleteDfsDir) { base_dir.delete(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 5cab978a13eea..35e7e6d3b9661 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.test.GenericTestUtils; import org.apache.log4j.Level; @@ -169,6 +168,7 @@ private byte getByte(long pos) { } private void testOneFile(String src, int writeBytes) throws IOException { + src += "_" + writeBytes; Path testPath = new Path(src); byte[] bytes = generateBytes(writeBytes); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index f65d0c750003c..e8e562b53adea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.StringUtils; import org.apache.log4j.Level; import org.junit.Assert; import org.junit.Test; @@ -71,6 +73,38 @@ public class TestDFSStripedOutputStreamWithFailure { private static final int FLUSH_POS = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1; + static { + System.out.println("NUM_DATA_BLOCKS = " + NUM_DATA_BLOCKS); + System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS); + System.out.println("CELL_SIZE = " + CELL_SIZE + + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(CELL_SIZE, "B", 2) + ")"); + System.out.println("BLOCK_SIZE = " + BLOCK_SIZE + + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_SIZE, "B", 2) + ")"); + System.out.println("BLOCK_GROUP_SIZE = " + BLOCK_GROUP_SIZE + + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_GROUP_SIZE, "B", 2) + ")"); + } + + static List newLengths() { + final List lengths = new ArrayList<>(); + lengths.add(FLUSH_POS + 2); + for(int b = 0; b <= 2; b++) { + for(int c = 0; c < STRIPES_PER_BLOCK*NUM_DATA_BLOCKS; c++) { + for(int delta = -1; delta <= 1; delta++) { + final int length = b*BLOCK_GROUP_SIZE + c*CELL_SIZE + delta; + System.out.println(lengths.size() + ": length=" + length + + ", (b, c, d) = (" + b + ", " + c + ", " + delta + ")"); + lengths.add(length); + } + } + } + return lengths; + } + + private static final List LENGTHS = newLengths(); + + static int getLength(int i) { + return LENGTHS.get(i); + } private MiniDFSCluster cluster; private DistributedFileSystem dfs; @@ -96,50 +130,49 @@ private static byte getByte(long pos) { return (byte)pos; } - private void initConf(Configuration conf){ + private HdfsConfiguration newHdfsConfiguration() { + final HdfsConfiguration conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); + return conf; } - private void initConfWithBlockToken(Configuration conf) { - conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); - conf.setInt("ipc.client.connect.max.retries", 0); - // Set short retry timeouts so this test runs faster - conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); - } - - @Test(timeout=240000) - public void testDatanodeFailure() throws Exception { - final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE); - HdfsConfiguration conf = new HdfsConfiguration(); - initConf(conf); + void runTest(final int length) { + final HdfsConfiguration conf = newHdfsConfiguration(); for (int dn = 0; dn < 9; dn++) { try { setup(conf); - cluster.startDataNodes(conf, 1, true, null, null); - cluster.waitActive(); - runTest(new Path(dir, "file" + dn), length, length / 2, dn, false); + runTest(length, dn, false, conf); } catch (Exception e) { - LOG.error("failed, dn=" + dn + ", length=" + length); - throw e; + final String err = "failed, dn=" + dn + ", length=" + length + + StringUtils.stringifyException(e); + LOG.error(err); + Assert.fail(err); } finally { tearDown(); } } } + @Test(timeout=240000) + public void testDatanodeFailure56() throws Exception { + runTest(getLength(56)); + } + @Test(timeout=240000) public void testBlockTokenExpired() throws Exception { final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE); - HdfsConfiguration conf = new HdfsConfiguration(); - initConf(conf); - initConfWithBlockToken(conf); + final HdfsConfiguration conf = newHdfsConfiguration(); + + conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0); + // Set short retry timeouts so this test runs faster + conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); for (int dn = 0; dn < 9; dn += 2) { try { setup(conf); - cluster.startDataNodes(conf, 1, true, null, null); - cluster.waitActive(); - runTest(new Path(dir, "file" + dn), length, length / 2, dn, true); + runTest(length, dn, true, conf); } catch (Exception e) { LOG.error("failed, dn=" + dn + ", length=" + length); throw e; @@ -229,19 +262,41 @@ public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException { } } - private void runTest(final Path p, final int length, final int killPos, - final int dnIndex, final boolean tokenExpire) throws Exception { - LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos - + ", dnIndex=" + dnIndex); - Preconditions.checkArgument(killPos < length); - Preconditions.checkArgument(killPos > FLUSH_POS); + private void runTest(final int length, final int dnIndex, + final boolean tokenExpire, final HdfsConfiguration conf) { + try { + runTest(length, length/2, dnIndex, tokenExpire, conf); + } catch(Exception e) { + LOG.info("FAILED", e); + Assert.fail(StringUtils.stringifyException(e)); + } + } + + private void runTest(final int length, final int killPos, + final int dnIndex, final boolean tokenExpire, + final HdfsConfiguration conf) throws Exception { + if (killPos <= FLUSH_POS) { + LOG.warn("killPos=" + killPos + " <= FLUSH_POS=" + FLUSH_POS + + ", length=" + length + ", dnIndex=" + dnIndex); + return; //skip test + } + Preconditions.checkArgument(length > killPos, + "length=%s <= killPos=%s", length, killPos); + + // start a datanode now, will kill one later + cluster.startDataNodes(conf, 1, true, null, null); + cluster.waitActive(); + + final Path p = new Path(dir, "dn" + dnIndex + "len" + length + "kill" + killPos); final String fullPath = p.toString(); + LOG.info("fullPath=" + fullPath); - final NameNode nn = cluster.getNameNode(); - final BlockManager bm = nn.getNamesystem().getBlockManager(); - final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); if (tokenExpire) { + final NameNode nn = cluster.getNameNode(); + final BlockManager bm = nn.getNamesystem().getBlockManager(); + final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); + // set a short token lifetime (1 second) SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); } @@ -265,7 +320,7 @@ private void runTest(final Path p, final int length, final int killPos, waitTokenExpires(out); } - StripedFileTestUtil.killDatanode(cluster, stripedOut, dnIndex, pos); + killDatanode(cluster, stripedOut, dnIndex, pos); killed = true; } @@ -301,6 +356,40 @@ static long getGenerationStamp(DFSStripedOutputStream out) } + static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { + for(;;) { + DatanodeInfo[] datanodes = streamer.getNodes(); + if (datanodes == null) { + // try peeking following block. + final LocatedBlock lb = streamer.peekFollowingBlock(); + if (lb != null) { + datanodes = lb.getLocations(); + } + } + + if (datanodes != null) { + Assert.assertEquals(1, datanodes.length); + Assert.assertNotNull(datanodes[0]); + return datanodes[0]; + } + + try { + Thread.sleep(100); + } catch (InterruptedException ie) { + Assert.fail(StringUtils.stringifyException(ie)); + return null; + } + } + } + + static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, + final int dnIndex, final AtomicInteger pos) { + final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex); + final DatanodeInfo datanode = getDatanodes(s); + LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos); + cluster.stopDataNode(datanode.getXferAddr()); + } + static void checkData(DistributedFileSystem dfs, String src, int length, int killedDnIndex, long oldGS) throws IOException { List> blockGroupList = new ArrayList<>(); @@ -314,7 +403,7 @@ static void checkData(DistributedFileSystem dfs, String src, int length, final long gs = firstBlock.getBlock().getGenerationStamp(); final String s = "gs=" + gs + ", oldGS=" + oldGS; LOG.info(s); - Assert.assertTrue(s, gs > oldGS); + Assert.assertTrue(s, gs >= oldGS); LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( (LocatedStripedBlock) firstBlock, @@ -342,7 +431,7 @@ static void checkData(DistributedFileSystem dfs, String src, int length, final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS + (j <= lastCellIndex? 1: 0); final int blockSize = numCellInBlock*CELL_SIZE - + (isLastGroup && i == lastCellIndex? lastCellSize - CELL_SIZE: 0); + + (isLastGroup && j == lastCellIndex? lastCellSize - CELL_SIZE: 0); final byte[] blockBytes = new byte[blockSize]; if (i < NUM_DATA_BLOCKS) { @@ -352,7 +441,8 @@ static void checkData(DistributedFileSystem dfs, String src, int length, } final LocatedBlock lb = blockList.get(i); - LOG.info("XXX i=" + i + ", lb=" + lb); + LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock + + ", blockSize=" + blockSize + ", lb=" + lb); if (lb == null) { continue; } @@ -410,4 +500,35 @@ private void waitTokenExpires(FSDataOutputStream out) throws IOException { } } } + + public static abstract class TestBase { + static final long TIMEOUT = 240000; + + int getBase() { + final String name = getClass().getSimpleName(); + int i = name.length() - 1; + for(; i >= 0 && Character.isDigit(name.charAt(i)); i--); + return Integer.parseInt(name.substring(i + 1)); + } + + private final TestDFSStripedOutputStreamWithFailure test + = new TestDFSStripedOutputStreamWithFailure(); + private void run(int offset) { + final int i = offset + getBase(); + final int length = getLength(i); + System.out.println("Run test " + i + ", length=" + length); + test.runTest(length); + } + + @Test(timeout=TIMEOUT) public void test0() {run(0);} + @Test(timeout=TIMEOUT) public void test1() {run(1);} + @Test(timeout=TIMEOUT) public void test2() {run(2);} + @Test(timeout=TIMEOUT) public void test3() {run(3);} + @Test(timeout=TIMEOUT) public void test4() {run(4);} + @Test(timeout=TIMEOUT) public void test5() {run(5);} + @Test(timeout=TIMEOUT) public void test6() {run(6);} + @Test(timeout=TIMEOUT) public void test7() {run(7);} + @Test(timeout=TIMEOUT) public void test8() {run(8);} + @Test(timeout=TIMEOUT) public void test9() {run(9);} + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java new file mode 100644 index 0000000000000..b4fb1b89d2950 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java @@ -0,0 +1,22 @@ +/** + * 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.hdfs.TestDFSStripedOutputStreamWithFailure.TestBase; + +public class TestDFSStripedOutputStreamWithFailure000 extends TestBase {} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java new file mode 100644 index 0000000000000..8489c3d62989d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java @@ -0,0 +1,22 @@ +/** + * 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.hdfs.TestDFSStripedOutputStreamWithFailure.TestBase; + +public class TestDFSStripedOutputStreamWithFailure010 extends TestBase {} \ No newline at end of file From 164cbe643988f878f0f4100a4de51783e5b6738e Mon Sep 17 00:00:00 2001 From: Walter Su Date: Thu, 27 Aug 2015 16:02:30 +0800 Subject: [PATCH 196/212] HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature. Contributed by Jing Zhao. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../server/blockmanagement/BlockInfo.java | 105 +++++-- .../blockmanagement/BlockInfoContiguous.java | 23 -- .../BlockInfoContiguousUnderConstruction.java | 281 ------------------ .../blockmanagement/BlockInfoStriped.java | 21 -- .../BlockInfoUnderConstruction.java | 84 ------ .../server/blockmanagement/BlockManager.java | 96 +++--- ...ava => BlockUnderConstructionFeature.java} | 138 ++++----- .../blockmanagement/DatanodeDescriptor.java | 10 +- .../blockmanagement/DatanodeManager.java | 24 +- .../hdfs/server/namenode/FSDirTruncateOp.java | 42 +-- .../server/namenode/FSDirWriteFileOp.java | 31 +- .../hdfs/server/namenode/FSEditLogLoader.java | 15 +- .../hdfs/server/namenode/FSImageFormat.java | 9 +- .../server/namenode/FSImageFormatPBINode.java | 9 +- .../server/namenode/FSImageSerialization.java | 6 +- .../hdfs/server/namenode/FSNamesystem.java | 40 ++- .../hdfs/server/namenode/INodeFile.java | 33 +- .../namenode/snapshot/FileDiffList.java | 3 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 9 +- .../TestBlockInfoUnderConstruction.java | 17 +- .../blockmanagement/TestBlockManager.java | 9 +- .../TestHeartbeatHandling.java | 21 +- .../TestReplicationPolicy.java | 12 +- .../server/namenode/TestAddStripedBlocks.java | 38 ++- .../namenode/TestBlockUnderConstruction.java | 6 +- .../TestCommitBlockSynchronization.java | 9 +- .../server/namenode/TestFileTruncate.java | 5 +- .../server/namenode/TestStripedINodeFile.java | 11 +- .../namenode/ha/TestRetryCacheWithHA.java | 11 +- .../namenode/snapshot/SnapshotTestHelper.java | 6 +- 31 files changed, 363 insertions(+), 764 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java rename hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/{BlockInfoStripedUnderConstruction.java => BlockUnderConstructionFeature.java} (68%) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8b25e68c04b92..28cc34aa922ad 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -403,3 +403,6 @@ HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream when the data length is small. (szetszwo via waltersu4549) + + HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC + to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549) 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 bf11914909868..f440e14035655 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 @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hdfs.server.blockmanagement; +import java.io.IOException; import java.util.LinkedList; +import com.google.common.base.Preconditions; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.util.LightWeightGSet; @@ -52,6 +54,8 @@ public abstract class BlockInfo extends Block */ protected Object[] triplets; + private BlockUnderConstructionFeature uc; + /** * Construct an entry for blocksmap * @param size the block's replication factor, or the total number of blocks @@ -287,26 +291,6 @@ public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage, return this; } - /** - * BlockInfo represents a block that is not being constructed. - * In order to start modifying the block, the BlockInfo should be converted to - * {@link BlockInfoContiguousUnderConstruction} or - * {@link BlockInfoStripedUnderConstruction}. - * @return {@link BlockUCState#COMPLETE} - */ - public BlockUCState getBlockUCState() { - return BlockUCState.COMPLETE; - } - - /** - * Is this block complete? - * - * @return true if the state of the block is {@link BlockUCState#COMPLETE} - */ - public boolean isComplete() { - return getBlockUCState().equals(BlockUCState.COMPLETE); - } - public boolean isDeleted() { return (bc == null); } @@ -332,4 +316,85 @@ public LightWeightGSet.LinkedElement getNext() { public void setNext(LightWeightGSet.LinkedElement next) { this.nextLinkedElement = next; } + + /* UnderConstruction Feature related */ + + public BlockUnderConstructionFeature getUnderConstructionFeature() { + return uc; + } + + public BlockUCState getBlockUCState() { + return uc == null ? BlockUCState.COMPLETE : uc.getBlockUCState(); + } + + /** + * Is this block complete? + * + * @return true if the state of the block is {@link BlockUCState#COMPLETE} + */ + public boolean isComplete() { + return getBlockUCState().equals(BlockUCState.COMPLETE); + } + + /** + * Add/Update the under construction feature. + */ + public void convertToBlockUnderConstruction(BlockUCState s, + DatanodeStorageInfo[] targets) { + if (isComplete()) { + uc = new BlockUnderConstructionFeature(this, s, targets, this.isStriped()); + } else { + // the block is already under construction + uc.setBlockUCState(s); + uc.setExpectedLocations(this, targets, this.isStriped()); + } + } + + /** + * Convert an under construction block to a complete block. + * + * @return BlockInfo - a complete block. + * @throws IOException if the state of the block + * (the generation stamp and the length) has not been committed by + * the client or it does not have at least a minimal number of replicas + * reported from data-nodes. + */ + BlockInfo convertToCompleteBlock() throws IOException { + assert getBlockUCState() != BlockUCState.COMPLETE : + "Trying to convert a COMPLETE block"; + uc = null; + return this; + } + + /** + * Process the recorded replicas. When about to commit or finish the + * pipeline recovery sort out bad replicas. + * @param genStamp The final generation stamp for the block. + */ + public void setGenerationStampAndVerifyReplicas(long genStamp) { + Preconditions.checkState(uc != null && !isComplete()); + // Set the generation stamp for the block. + setGenerationStamp(genStamp); + + // Remove the replicas with wrong gen stamp + uc.removeStaleReplicas(this); + } + + /** + * Commit block's length and generation stamp as reported by the client. + * Set block state to {@link BlockUCState#COMMITTED}. + * @param block - contains client reported block length and generation + * @throws IOException if block ids are inconsistent. + */ + void commitBlock(Block block) throws IOException { + if (getBlockId() != block.getBlockId()) { + throw new IOException("Trying to commit inconsistent block: id = " + + block.getBlockId() + ", expected id = " + getBlockId()); + } + Preconditions.checkState(!isComplete()); + uc.commit(); + this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp()); + // Sort out invalid replicas. + setGenerationStampAndVerifyReplicas(block.getGenerationStamp()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index bb9bf5b27bd8a..12b4fd3f95cad 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -19,7 +19,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; /** * Subclass of {@link BlockInfo}, used for a block with replication scheme. @@ -123,28 +122,6 @@ void replaceBlock(BlockInfo newBlock) { } } - /** - * Convert a complete block to an under construction block. - * @return BlockInfoUnderConstruction - an under construction block. - */ - public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction( - BlockUCState s, DatanodeStorageInfo[] targets) { - if(isComplete()) { - BlockInfoContiguousUnderConstruction ucBlock = - new BlockInfoContiguousUnderConstruction(this, - getBlockCollection().getPreferredBlockReplication(), s, targets); - ucBlock.setBlockCollection(getBlockCollection()); - return ucBlock; - } - // the block is already under construction - BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction) this; - ucBlock.setBlockUCState(s); - ucBlock.setExpectedLocations(targets); - ucBlock.setBlockCollection(getBlockCollection()); - return ucBlock; - } - @Override public final boolean isStriped() { return false; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java deleted file mode 100644 index 96b209def6d2b..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java +++ /dev/null @@ -1,281 +0,0 @@ -/** - * 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.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; -import org.apache.hadoop.hdfs.server.namenode.NameNode; - -/** - * Represents a block that is currently being constructed.
    - * This is usually the last block of a file opened for write or append. - */ -public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous - implements BlockInfoUnderConstruction{ - /** Block state. See {@link BlockUCState} */ - private BlockUCState blockUCState; - - /** - * Block replicas as assigned when the block was allocated. - * This defines the pipeline order. - */ - private List replicas; - - /** - * Index of the primary data node doing the recovery. Useful for log - * messages. - */ - private int primaryNodeIndex = -1; - - /** - * The new generation stamp, which this block will have - * after the recovery succeeds. Also used as a recovery id to identify - * the right recovery if any of the abandoned recoveries re-appear. - */ - private long blockRecoveryId = 0; - - /** - * The block source to use in the event of copy-on-write truncate. - */ - private Block truncateBlock; - - /** - * Create block and set its state to - * {@link BlockUCState#UNDER_CONSTRUCTION}. - */ - public BlockInfoContiguousUnderConstruction(Block blk, short replication) { - this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null); - } - - /** - * Create a block that is currently being constructed. - */ - public BlockInfoContiguousUnderConstruction(Block blk, short replication, - BlockUCState state, DatanodeStorageInfo[] targets) { - super(blk, replication); - assert getBlockUCState() != BlockUCState.COMPLETE : - "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state"; - this.blockUCState = state; - setExpectedLocations(targets); - } - - @Override - public BlockInfoContiguous convertToCompleteBlock() throws IOException { - assert getBlockUCState() != BlockUCState.COMPLETE : - "Trying to convert a COMPLETE block"; - return new BlockInfoContiguous(this); - } - - @Override - public void setExpectedLocations(DatanodeStorageInfo[] targets) { - int numLocations = targets == null ? 0 : targets.length; - this.replicas = new ArrayList<>(numLocations); - for(int i = 0; i < numLocations; i++) { - replicas.add(new ReplicaUnderConstruction(this, targets[i], - ReplicaState.RBW)); - } - } - - @Override - public DatanodeStorageInfo[] getExpectedStorageLocations() { - int numLocations = replicas == null ? 0 : replicas.size(); - DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; - for (int i = 0; i < numLocations; i++) { - storages[i] = replicas.get(i).getExpectedStorageLocation(); - } - return storages; - } - - @Override - public int getNumExpectedLocations() { - return replicas == null ? 0 : replicas.size(); - } - - /** - * Return the state of the block under construction. - * @see BlockUCState - */ - @Override // BlockInfo - public BlockUCState getBlockUCState() { - return blockUCState; - } - - void setBlockUCState(BlockUCState s) { - blockUCState = s; - } - - @Override - public long getBlockRecoveryId() { - return blockRecoveryId; - } - - @Override - public Block getTruncateBlock() { - return truncateBlock; - } - - @Override - public Block toBlock(){ - return this; - } - - public void setTruncateBlock(Block recoveryBlock) { - this.truncateBlock = recoveryBlock; - } - - @Override - public void setGenerationStampAndVerifyReplicas(long genStamp) { - // Set the generation stamp for the block. - setGenerationStamp(genStamp); - if (replicas == null) - return; - - // Remove the replicas with wrong gen stamp. - // The replica list is unchanged. - for (ReplicaUnderConstruction r : replicas) { - if (genStamp != r.getGenerationStamp()) { - r.getExpectedStorageLocation().removeBlock(this); - NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica " - + "from location: {}", r.getExpectedStorageLocation()); - } - } - } - - @Override - public void commitBlock(Block block) throws IOException { - if(getBlockId() != block.getBlockId()) - throw new IOException("Trying to commit inconsistent block: id = " - + block.getBlockId() + ", expected id = " + getBlockId()); - blockUCState = BlockUCState.COMMITTED; - this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp()); - // Sort out invalid replicas. - setGenerationStampAndVerifyReplicas(block.getGenerationStamp()); - } - - @Override - public void initializeBlockRecovery(long recoveryId) { - setBlockUCState(BlockUCState.UNDER_RECOVERY); - blockRecoveryId = recoveryId; - if (replicas.size() == 0) { - NameNode.blockStateChangeLog.warn("BLOCK*" - + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:" - + " No blocks found, lease removed."); - } - boolean allLiveReplicasTriedAsPrimary = true; - for (ReplicaUnderConstruction replica : replicas) { - // Check if all replicas have been tried or not. - if (replica.isAlive()) { - allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary && - replica.getChosenAsPrimary()); - } - } - if (allLiveReplicasTriedAsPrimary) { - // Just set all the replicas to be chosen whether they are alive or not. - for (ReplicaUnderConstruction replica : replicas) { - replica.setChosenAsPrimary(false); - } - } - long mostRecentLastUpdate = 0; - ReplicaUnderConstruction primary = null; - primaryNodeIndex = -1; - for(int i = 0; i < replicas.size(); i++) { - // Skip alive replicas which have been chosen for recovery. - if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) { - continue; - } - final ReplicaUnderConstruction ruc = replicas.get(i); - final long lastUpdate = ruc.getExpectedStorageLocation() - .getDatanodeDescriptor().getLastUpdateMonotonic(); - if (lastUpdate > mostRecentLastUpdate) { - primaryNodeIndex = i; - primary = ruc; - mostRecentLastUpdate = lastUpdate; - } - } - if (primary != null) { - primary.getExpectedStorageLocation().getDatanodeDescriptor() - .addBlockToBeRecovered(this); - primary.setChosenAsPrimary(true); - NameNode.blockStateChangeLog.debug( - "BLOCK* {} recovery started, primary={}", this, primary); - } - } - - @Override - public void addReplicaIfNotPresent(DatanodeStorageInfo storage, - Block block, ReplicaState rState) { - Iterator it = replicas.iterator(); - while (it.hasNext()) { - ReplicaUnderConstruction r = it.next(); - DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation(); - if(expectedLocation == storage) { - // Record the gen stamp from the report - r.setGenerationStamp(block.getGenerationStamp()); - return; - } else if (expectedLocation != null && - expectedLocation.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, storage, rState)); - } - - @Override - public String toString() { - final StringBuilder b = new StringBuilder(100); - appendStringTo(b); - return b.toString(); - } - - @Override - public void appendStringTo(StringBuilder sb) { - super.appendStringTo(sb); - appendUCParts(sb); - } - - private void appendUCParts(StringBuilder sb) { - sb.append("{UCState=").append(blockUCState) - .append(", truncateBlock=" + truncateBlock) - .append(", primaryNodeIndex=").append(primaryNodeIndex) - .append(", replicas=["); - if (replicas != null) { - Iterator iter = replicas.iterator(); - if (iter.hasNext()) { - iter.next().appendStringTo(sb); - while (iter.hasNext()) { - sb.append(", "); - iter.next().appendStringTo(sb); - } - } - } - sb.append("]}"); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 14d2fcc0e722a..60937764b3f90 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -245,27 +245,6 @@ public int numNodes() { return num; } - /** - * Convert a complete block to an under construction block. - * @return BlockInfoUnderConstruction - an under construction block. - */ - public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( - BlockUCState s, DatanodeStorageInfo[] targets) { - final BlockInfoStripedUnderConstruction ucBlock; - if(isComplete()) { - ucBlock = new BlockInfoStripedUnderConstruction(this, ecPolicy, - s, targets); - ucBlock.setBlockCollection(getBlockCollection()); - } else { - // the block is already under construction - ucBlock = (BlockInfoStripedUnderConstruction) this; - ucBlock.setBlockUCState(s); - ucBlock.setExpectedLocations(targets); - ucBlock.setBlockCollection(getBlockCollection()); - } - return ucBlock; - } - @Override final boolean hasNoStorage() { final int len = getCapacity(); 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 deleted file mode 100644 index 10a8caeae3c4e..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java +++ /dev/null @@ -1,84 +0,0 @@ -/** - * 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.io.IOException; - -import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; - -public interface BlockInfoUnderConstruction { - /** - * Create array of expected replica locations - * (as has been assigned by chooseTargets()). - */ - public DatanodeStorageInfo[] getExpectedStorageLocations(); - - /** Get recover block */ - public Block getTruncateBlock(); - - /** Convert to a Block object */ - public Block toBlock(); - - /** Get block recovery ID */ - public long getBlockRecoveryId(); - - /** Get the number of expected locations */ - public int getNumExpectedLocations(); - - /** Set expected locations */ - public void setExpectedLocations(DatanodeStorageInfo[] targets); - - /** - * Process the recorded replicas. When about to commit or finish the - * pipeline recovery sort out bad replicas. - * @param genStamp The final generation stamp for the block. - */ - public void setGenerationStampAndVerifyReplicas(long genStamp); - - /** - * Initialize lease recovery for this block. - * Find the first alive data-node starting from the previous primary and - * make it primary. - */ - public void initializeBlockRecovery(long recoveryId); - - /** Add the reported replica if it is not already in the replica list. */ - public void addReplicaIfNotPresent(DatanodeStorageInfo storage, - Block reportedBlock, ReplicaState rState); - - /** - * Commit block's length and generation stamp as reported by the client. - * Set block state to {@link BlockUCState#COMMITTED}. - * @param block - contains client reported block length and generation - * @throws IOException if block ids are inconsistent. - */ - public void commitBlock(Block block) throws IOException; - - /** - * Convert an under construction block to a complete block. - * - * @return a complete block. - * @throws IOException - * if the state of the block (the generation stamp and the length) - * has not been committed by the client or it does not have at least - * a minimal number of replicas reported from data-nodes. - */ - public BlockInfo convertToCompleteBlock() throws IOException; -} 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 dfea5f3b93de7..ae088256c5311 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 @@ -644,19 +644,13 @@ public boolean hasMinStorage(BlockInfo block, int liveNum) { */ private static boolean commitBlock(final BlockInfo block, final Block commitBlock) throws IOException { - if (block instanceof BlockInfoUnderConstruction - && block.getBlockUCState() != BlockUCState.COMMITTED) { - final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block; - - assert block.getNumBytes() <= commitBlock.getNumBytes() : + if (block.getBlockUCState() == BlockUCState.COMMITTED) + return false; + assert block.getNumBytes() <= commitBlock.getNumBytes() : "commitBlock length is less than the stored one " - + commitBlock.getNumBytes() + " vs. " + block.getNumBytes(); - - uc.commitBlock(commitBlock); - return true; - } - - return false; + + commitBlock.getNumBytes() + " vs. " + block.getNumBytes(); + block.commitBlock(commitBlock); + return true; } /** @@ -713,9 +707,7 @@ private BlockInfo completeBlock(final BlockCollection bc, "Cannot complete block: block has not been COMMITTED by the client"); } - final BlockInfo completeBlock - = !(curBlock instanceof BlockInfoUnderConstruction)? curBlock - : ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock(); + final BlockInfo completeBlock = curBlock.convertToCompleteBlock(); // replace penultimate block in file bc.setBlock(blkIndex, completeBlock); @@ -754,9 +746,7 @@ private BlockInfo completeBlock(final BlockCollection bc, */ public BlockInfo forceCompleteBlock(final BlockCollection bc, final BlockInfo block) throws IOException { - if (block instanceof BlockInfoUnderConstruction) { - ((BlockInfoUnderConstruction)block).commitBlock(block); - } + block.commitBlock(block); return completeBlock(bc, block, true); } @@ -777,30 +767,28 @@ public BlockInfo forceCompleteBlock(final BlockCollection bc, */ public LocatedBlock convertLastBlockToUnderConstruction( BlockCollection bc, long bytesToRemove) throws IOException { - BlockInfo oldBlock = bc.getLastBlock(); - if(oldBlock == null || - bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove) + BlockInfo lastBlock = bc.getLastBlock(); + if(lastBlock == null || + bc.getPreferredBlockSize() == lastBlock.getNumBytes() - bytesToRemove) return null; - assert oldBlock == getStoredBlock(oldBlock) : + assert lastBlock == getStoredBlock(lastBlock) : "last block of the file is not in blocksMap"; - DatanodeStorageInfo[] targets = getStorages(oldBlock); + DatanodeStorageInfo[] targets = getStorages(lastBlock); - // convert the last block to UC - bc.convertLastBlockToUC(oldBlock, targets); - // get the new created uc block - BlockInfo ucBlock = bc.getLastBlock(); - blocksMap.replaceBlock(ucBlock); + // convert the last block to under construction. note no block replacement + // is happening + bc.convertLastBlockToUC(lastBlock, targets); // Remove block from replication queue. - NumberReplicas replicas = countNodes(ucBlock); - neededReplications.remove(ucBlock, replicas.liveReplicas(), - replicas.decommissionedAndDecommissioning(), getReplication(ucBlock)); - pendingReplications.remove(ucBlock); + NumberReplicas replicas = countNodes(lastBlock); + neededReplications.remove(lastBlock, replicas.liveReplicas(), + replicas.decommissionedAndDecommissioning(), getReplication(lastBlock)); + pendingReplications.remove(lastBlock); // remove this block from the list of pending blocks to be deleted. for (DatanodeStorageInfo storage : targets) { - final Block b = getBlockOnStorage(oldBlock, storage); + final Block b = getBlockOnStorage(lastBlock, storage); if (b != null) { invalidateBlocks.remove(storage.getDatanodeDescriptor(), b); } @@ -810,13 +798,15 @@ public LocatedBlock convertLastBlockToUnderConstruction( // count in safe-mode. namesystem.adjustSafeModeBlockTotals( // decrement safe if we had enough - hasMinStorage(oldBlock, targets.length) ? -1 : 0, + hasMinStorage(lastBlock, targets.length) ? -1 : 0, // always decrement total blocks -1); - final long fileLength = bc.computeContentSummary(getStoragePolicySuite()).getLength(); - final long pos = fileLength - ucBlock.getNumBytes(); - return createLocatedBlock(ucBlock, pos, BlockTokenIdentifier.AccessMode.WRITE); + final long fileLength = bc.computeContentSummary( + getStoragePolicySuite()).getLength(); + final long pos = fileLength - lastBlock.getNumBytes(); + return createLocatedBlock(lastBlock, pos, + BlockTokenIdentifier.AccessMode.WRITE); } /** @@ -895,18 +885,14 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos, private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) throws IOException { if (!blk.isComplete()) { + final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature(); if (blk.isStriped()) { - final BlockInfoStripedUnderConstruction uc = - (BlockInfoStripedUnderConstruction) blk; final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, false); } else { - assert blk instanceof BlockInfoContiguousUnderConstruction; - final BlockInfoContiguousUnderConstruction uc = - (BlockInfoContiguousUnderConstruction) blk; final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); @@ -1923,9 +1909,7 @@ static class StatefulBlockInfo { StatefulBlockInfo(BlockInfo storedBlock, Block reportedBlock, ReplicaState reportedState) { - Preconditions.checkArgument( - storedBlock instanceof BlockInfoContiguousUnderConstruction || - storedBlock instanceof BlockInfoStripedUnderConstruction); + Preconditions.checkArgument(!storedBlock.isComplete()); this.storedBlock = storedBlock; this.reportedBlock = reportedBlock; this.reportedState = reportedState; @@ -2335,13 +2319,14 @@ private void processFirstBlockReport( // If block is under construction, add this replica to its list if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) { - final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)storedBlock; - uc.addReplicaIfNotPresent(storageInfo, iblk, reportedState); + storedBlock.getUnderConstructionFeature() + .addReplicaIfNotPresent(storageInfo, 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 if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) { - int numOfReplicas = uc.getNumExpectedLocations(); + int numOfReplicas = storedBlock.getUnderConstructionFeature() + .getNumExpectedLocations(); namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock); } //and fall through to next clause @@ -2469,11 +2454,6 @@ private BlockInfo processReportedBlock( // Ignore replicas already scheduled to be removed from the DN if(invalidateBlocks.contains(dn, block)) { - /* - * TODO: following assertion is incorrect, see HDFS-2668 assert - * storedBlock.findDatanode(dn) < 0 : "Block " + block + - * " in recentInvalidatesSet should not appear in DN " + dn; - */ return storedBlock; } @@ -2704,9 +2684,8 @@ private boolean isBlockUnderConstruction(BlockInfo storedBlock, void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, DatanodeStorageInfo storageInfo) throws IOException { BlockInfo block = ucBlock.storedBlock; - final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block; - uc.addReplicaIfNotPresent(storageInfo, ucBlock.reportedBlock, - ucBlock.reportedState); + block.getUnderConstructionFeature().addReplicaIfNotPresent(storageInfo, + ucBlock.reportedBlock, ucBlock.reportedState); if (ucBlock.reportedState == ReplicaState.FINALIZED && (block.findStorageInfo(storageInfo) < 0)) { @@ -2766,8 +2745,7 @@ private Block addStoredBlock(final BlockInfo block, assert block != null && namesystem.hasWriteLock(); BlockInfo storedBlock; DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); - if (block instanceof BlockInfoContiguousUnderConstruction || - block instanceof BlockInfoStripedUnderConstruction) { + if (!block.isComplete()) { //refresh our copy in case the block got completed in another thread storedBlock = getStoredBlock(block); } else { @@ -4221,7 +4199,7 @@ public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, final LocatedBlock lb; if (info.isStriped()) { lb = newLocatedStripedBlock(eb, locs, - ((BlockInfoStripedUnderConstruction)info).getBlockIndices(), + info.getUnderConstructionFeature().getBlockIndices(), offset, false); } else { lb = newLocatedBlock(eb, locs, offset, false); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java similarity index 68% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java index 9de8294c82e9d..58b455e1f1d14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java @@ -21,19 +21,14 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.namenode.NameNode; -import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; - -import java.io.IOException; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE; -import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION; /** - * Represents a striped block that is currently being constructed. + * Represents the under construction feature of a Block. * This is usually the last block of a file opened for write or append. */ -public class BlockInfoStripedUnderConstruction extends BlockInfoStriped - implements BlockInfoUnderConstruction{ +public class BlockUnderConstructionFeature { private BlockUCState blockUCState; /** @@ -55,41 +50,30 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped private long blockRecoveryId = 0; /** - * Constructor with null storage targets. + * The block source to use in the event of copy-on-write truncate. */ - public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy) { - this(blk, ecPolicy, UNDER_CONSTRUCTION, null); - } + private Block truncateBlock; - /** - * Create a striped block that is currently being constructed. - */ - public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy, - BlockUCState state, DatanodeStorageInfo[] targets) { - super(blk, ecPolicy); + public BlockUnderConstructionFeature(Block blk, + BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) { assert getBlockUCState() != COMPLETE : - "BlockInfoStripedUnderConstruction cannot be in COMPLETE state"; + "BlockUnderConstructionFeature cannot be in COMPLETE state"; this.blockUCState = state; - setExpectedLocations(targets); - } - - @Override - public BlockInfoStriped convertToCompleteBlock() throws IOException { - assert getBlockUCState() != COMPLETE : - "Trying to convert a COMPLETE block"; - return new BlockInfoStriped(this); + setExpectedLocations(blk, targets, isStriped); } /** Set expected locations */ - @Override - public void setExpectedLocations(DatanodeStorageInfo[] targets) { + public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets, + boolean isStriped) { int numLocations = targets == null ? 0 : targets.length; this.replicas = new ReplicaUnderConstruction[numLocations]; for(int i = 0; i < numLocations; i++) { - // when creating a new block we simply sequentially assign block index to - // each storage - Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp()); - replicas[i] = new ReplicaUnderConstruction(blk, targets[i], + // when creating a new striped block we simply sequentially assign block + // index to each storage + Block replicaBlock = isStriped ? + new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) : + block; + replicas[i] = new ReplicaUnderConstruction(replicaBlock, targets[i], ReplicaState.RBW); } } @@ -98,7 +82,6 @@ public void setExpectedLocations(DatanodeStorageInfo[] targets) { * Create array of expected replica locations * (as has been assigned by chooseTargets()). */ - @Override public DatanodeStorageInfo[] getExpectedStorageLocations() { int numLocations = getNumExpectedLocations(); DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations]; @@ -108,7 +91,10 @@ public DatanodeStorageInfo[] getExpectedStorageLocations() { return storages; } - /** @return the index array indicating the block index in each storage */ + /** + * @return the index array indicating the block index in each storage. Used + * only by striped blocks. + */ public int[] getBlockIndices() { int numLocations = getNumExpectedLocations(); int[] indices = new int[numLocations]; @@ -118,7 +104,6 @@ public int[] getBlockIndices() { return indices; } - @Override public int getNumExpectedLocations() { return replicas == null ? 0 : replicas.length; } @@ -127,7 +112,6 @@ public int getNumExpectedLocations() { * Return the state of the block under construction. * @see BlockUCState */ - @Override // BlockInfo public BlockUCState getBlockUCState() { return blockUCState; } @@ -136,58 +120,51 @@ void setBlockUCState(BlockUCState s) { blockUCState = s; } - @Override public long getBlockRecoveryId() { return blockRecoveryId; } - @Override + /** Get recover block */ public Block getTruncateBlock() { - return null; + return truncateBlock; } - @Override - public Block toBlock(){ - return this; + public void setTruncateBlock(Block recoveryBlock) { + this.truncateBlock = recoveryBlock; } - @Override - public void setGenerationStampAndVerifyReplicas(long genStamp) { - // Set the generation stamp for the block. - setGenerationStamp(genStamp); - if (replicas == null) - return; - - // Remove the replicas with wrong gen stamp. - // The replica list is unchanged. - for (ReplicaUnderConstruction r : replicas) { - if (genStamp != r.getGenerationStamp()) { - r.getExpectedStorageLocation().removeBlock(this); - NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica " - + "from location: {}", r.getExpectedStorageLocation()); - } - } + /** + * Set {@link #blockUCState} to {@link BlockUCState#COMMITTED}. + */ + void commit() { + blockUCState = BlockUCState.COMMITTED; } - @Override - public void commitBlock(Block block) throws IOException { - if (getBlockId() != block.getBlockId()) { - throw new IOException("Trying to commit inconsistent block: id = " - + block.getBlockId() + ", expected id = " + getBlockId()); + void removeStaleReplicas(BlockInfo block) { + final long genStamp = block.getGenerationStamp(); + if (replicas != null) { + // Remove replicas with wrong gen stamp. The replica list is unchanged. + for (ReplicaUnderConstruction r : replicas) { + if (genStamp != r.getGenerationStamp()) { + r.getExpectedStorageLocation().removeBlock(block); + NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica " + + "from location: {}", r.getExpectedStorageLocation()); + } + } } - blockUCState = BlockUCState.COMMITTED; - this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp()); - // Sort out invalid replicas. - setGenerationStampAndVerifyReplicas(block.getGenerationStamp()); } - @Override - public void initializeBlockRecovery(long recoveryId) { + /** + * Initialize lease recovery for this block. + * Find the first alive data-node starting from the previous primary and + * make it primary. + */ + public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) { setBlockUCState(BlockUCState.UNDER_RECOVERY); blockRecoveryId = recoveryId; if (replicas == null || replicas.length == 0) { NameNode.blockStateChangeLog.warn("BLOCK*" + - " BlockInfoStripedUnderConstruction.initLeaseRecovery:" + + " BlockUnderConstructionFeature.initLeaseRecovery:" + " No blocks found, lease removed."); // sets primary node index and return. primaryNodeIndex = -1; @@ -226,15 +203,15 @@ public void initializeBlockRecovery(long recoveryId) { } if (primary != null) { primary.getExpectedStorageLocation().getDatanodeDescriptor() - .addBlockToBeRecovered(this); + .addBlockToBeRecovered(blockInfo); primary.setChosenAsPrimary(true); NameNode.blockStateChangeLog.info( "BLOCK* {} recovery started, primary={}", this, primary); } } - @Override - public void addReplicaIfNotPresent(DatanodeStorageInfo storage, + /** Add the reported replica if it is not already in the replica list. */ + void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock, ReplicaState rState) { if (replicas == null) { replicas = new ReplicaUnderConstruction[1]; @@ -269,20 +246,15 @@ public void addReplicaIfNotPresent(DatanodeStorageInfo storage, @Override public String toString() { final StringBuilder b = new StringBuilder(100); - appendStringTo(b); + appendUCParts(b); return b.toString(); } - @Override - public void appendStringTo(StringBuilder sb) { - super.appendStringTo(sb); - appendUCParts(sb); - } - private void appendUCParts(StringBuilder sb) { - sb.append("{UCState=").append(blockUCState). - append(", primaryNodeIndex=").append(primaryNodeIndex). - append(", replicas=["); + sb.append("{UCState=").append(blockUCState) + .append(", truncateBlock=").append(truncateBlock) + .append(", primaryNodeIndex=").append(primaryNodeIndex) + .append(", replicas=["); if (replicas != null) { int i = 0; for (ReplicaUnderConstruction r : replicas) { 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 87394f649fe28..a4d54421d010c 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 @@ -227,7 +227,7 @@ public CachedBlocksList getPendingUncached() { private final BlockQueue erasurecodeBlocks = new BlockQueue<>(); /** A queue of blocks to be recovered by this datanode */ - private final BlockQueue recoverBlocks = + private final BlockQueue recoverBlocks = new BlockQueue<>(); /** A set of blocks to be invalidated by this datanode */ private final LightWeightHashSet invalidateBlocks = @@ -624,7 +624,7 @@ void addBlockToBeErasureCoded(ExtendedBlock block, /** * Store block recovery work. */ - void addBlockToBeRecovered(BlockInfoUnderConstruction block) { + void addBlockToBeRecovered(BlockInfo block) { if(recoverBlocks.contains(block)) { // this prevents adding the same block twice to the recovery queue BlockManager.LOG.info(block + " is already in the recovery queue"); @@ -678,11 +678,11 @@ public List getErasureCodeCommand(int maxTransfers) { return erasurecodeBlocks.poll(maxTransfers); } - public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { - List blocks = recoverBlocks.poll(maxTransfers); + public BlockInfo[] getLeaseRecoveryCommand(int maxTransfers) { + List blocks = recoverBlocks.poll(maxTransfers); if(blocks == null) return null; - return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]); + return blocks.toArray(new BlockInfo[blocks.size()]); } /** 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 dbd07d48b660a..8d2e750bfd290 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 @@ -1381,13 +1381,15 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, } //check lease recovery - BlockInfoUnderConstruction[] blocks = nodeinfo - .getLeaseRecoveryCommand(Integer.MAX_VALUE); + BlockInfo[] blocks = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE); if (blocks != null) { BlockRecoveryCommand brCommand = new BlockRecoveryCommand( blocks.length); - for (BlockInfoUnderConstruction b : blocks) { - final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations(); + for (BlockInfo b : blocks) { + final BlockUnderConstructionFeature uc = + b.getUnderConstructionFeature(); + assert uc != null; + final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); // Skip stale nodes during recovery - not heart beated for some time (30s by default). final List recoveryLocations = new ArrayList<>(storages.length); @@ -1398,12 +1400,12 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, } // If we are performing a truncate recovery than set recovery fields // to old block. - boolean truncateRecovery = b.getTruncateBlock() != null; + boolean truncateRecovery = uc.getTruncateBlock() != null; boolean copyOnTruncateRecovery = truncateRecovery && - b.getTruncateBlock().getBlockId() != b.toBlock().getBlockId(); + uc.getTruncateBlock().getBlockId() != b.getBlockId(); ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ? - new ExtendedBlock(blockPoolId, b.getTruncateBlock()) : - new ExtendedBlock(blockPoolId, b.toBlock()); + new ExtendedBlock(blockPoolId, uc.getTruncateBlock()) : + new ExtendedBlock(blockPoolId, b); // If we only get 1 replica after eliminating stale nodes, then choose all // replicas for recovery and let the primary data node handle failures. DatanodeInfo[] recoveryInfos; @@ -1420,13 +1422,13 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages); } if(truncateRecovery) { - Block recoveryBlock = (copyOnTruncateRecovery) ? b.toBlock() : - b.getTruncateBlock(); + Block recoveryBlock = (copyOnTruncateRecovery) ? b : + uc.getTruncateBlock(); brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos, recoveryBlock)); } else { brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos, - b.getBlockRecoveryId())); + uc.getBlockRecoveryId())); } } return new DatanodeCommand[] { brCommand }; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java index 215a7611e5614..3d5da8ebb633a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java @@ -28,8 +28,9 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; @@ -102,7 +103,7 @@ static TruncateResult truncate(final FSNamesystem fsn, final String srcArg, final BlockInfo last = file.getLastBlock(); if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { - final Block truncatedBlock = ((BlockInfoContiguousUnderConstruction) last) + final Block truncatedBlock = last.getUnderConstructionFeature() .getTruncateBlock(); if (truncatedBlock != null) { final long truncateLength = file.computeFileSize(false, false) @@ -231,43 +232,42 @@ static Block prepareFileForTruncate(FSNamesystem fsn, INodesInPath iip, oldBlock))); } - BlockInfoContiguousUnderConstruction truncatedBlockUC; + final BlockInfo truncatedBlockUC; BlockManager blockManager = fsn.getFSDirectory().getBlockManager(); if (shouldCopyOnTruncate) { // Add new truncateBlock into blocksMap and // use oldBlock as a source for copy-on-truncate recovery - truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock, + truncatedBlockUC = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); + truncatedBlockUC.convertToBlockUnderConstruction( + BlockUCState.UNDER_CONSTRUCTION, blockManager.getStorages(oldBlock)); truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta); - truncatedBlockUC.setTruncateBlock(oldBlock); - file.convertLastBlockToUC(truncatedBlockUC, - blockManager.getStorages(oldBlock)); + truncatedBlockUC.getUnderConstructionFeature().setTruncateBlock(oldBlock); + file.setLastBlock(truncatedBlockUC); blockManager.addBlockCollection(truncatedBlockUC, file); NameNode.stateChangeLog.debug( "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new" + " size {} new block {} old block {}", - truncatedBlockUC.getNumBytes(), newBlock, - truncatedBlockUC.getTruncateBlock()); + truncatedBlockUC.getNumBytes(), newBlock, oldBlock); } else { // Use new generation stamp for in-place truncate recovery blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta); oldBlock = file.getLastBlock(); assert !oldBlock.isComplete() : "oldBlock should be under construction"; - truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock; - truncatedBlockUC.setTruncateBlock(new Block(oldBlock)); - truncatedBlockUC.getTruncateBlock().setNumBytes( - oldBlock.getNumBytes() - lastBlockDelta); - truncatedBlockUC.getTruncateBlock().setGenerationStamp( - newBlock.getGenerationStamp()); - - NameNode.stateChangeLog.debug( - "BLOCK* prepareFileForTruncate: {} Scheduling in-place block " - + "truncate to new size {}", truncatedBlockUC.getTruncateBlock() - .getNumBytes(), truncatedBlockUC); + BlockUnderConstructionFeature uc = oldBlock.getUnderConstructionFeature(); + uc.setTruncateBlock(new Block(oldBlock)); + uc.getTruncateBlock().setNumBytes(oldBlock.getNumBytes() - lastBlockDelta); + uc.getTruncateBlock().setGenerationStamp(newBlock.getGenerationStamp()); + truncatedBlockUC = oldBlock; + + NameNode.stateChangeLog.debug("BLOCK* prepareFileForTruncate: " + + "{} Scheduling in-place block truncate to new size {}", + uc, uc.getTruncateBlock().getNumBytes()); } if (shouldRecoverNow) { - truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp()); + truncatedBlockUC.getUnderConstructionFeature().initializeBlockRecovery( + truncatedBlockUC, newBlock.getGenerationStamp()); } return newBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index ffd8fbcddfdf0..68aef7630c171 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -45,10 +45,10 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; @@ -77,7 +77,7 @@ static boolean unprotectedRemoveBlock( Block block) throws IOException { // modify file-> block and blocksMap // fileNode should be under construction - BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block); + BlockInfo uc = fileNode.removeLastBlock(block); if (uc == null) { return false; } @@ -214,8 +214,8 @@ static ValidateAddBlockResult validateAddBlock( static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk, DatanodeStorageInfo[] locs, long offset) throws IOException { - LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk), - blk, locs, offset); + LocatedBlock lBlk = BlockManager.newLocatedBlock( + fsn.getExtendedBlock(new Block(blk)), blk, locs, offset); fsn.getBlockManager().setBlockToken(lBlk, BlockTokenIdentifier.AccessMode.WRITE); return lBlk; @@ -247,8 +247,8 @@ static LocatedBlock storeAllocatedBlock(FSNamesystem fsn, String src, } else { // add new chosen targets to already allocated block and return BlockInfo lastBlockInFile = pendingFile.getLastBlock(); - ((BlockInfoContiguousUnderConstruction) lastBlockInFile) - .setExpectedLocations(targets); + lastBlockInFile.getUnderConstructionFeature().setExpectedLocations( + lastBlockInFile, targets, pendingFile.isStriped()); offset = pendingFile.computeFileSize(); return makeLocatedBlock(fsn, lastBlockInFile, targets, offset); } @@ -542,7 +542,8 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, // check quota limits and updated space consumed fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), numLocations, true); - blockInfo = new BlockInfoStripedUnderConstruction(block, ecPolicy, + blockInfo = new BlockInfoStriped(block, ecPolicy); + blockInfo.convertToBlockUnderConstruction( HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } else { // check quota limits and updated space consumed @@ -550,9 +551,9 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, fileINode.getPreferredBlockReplication(), true); short numLocations = fileINode.getFileReplication(); - blockInfo = new BlockInfoContiguousUnderConstruction(block, - numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, - targets); + blockInfo = new BlockInfoContiguous(block, numLocations); + blockInfo.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); } fsd.getBlockManager().addBlockCollection(blockInfo, fileINode); fileINode.addBlock(blockInfo); @@ -692,10 +693,10 @@ private static FileState analyzeFileState( "allocation of a new block in " + src + ". Returning previously" + " allocated block " + lastBlockInFile); long offset = file.computeFileSize(); - BlockInfoUnderConstruction lastBlockUC = - (BlockInfoUnderConstruction) lastBlockInFile; + BlockUnderConstructionFeature uc = + lastBlockInFile.getUnderConstructionFeature(); onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile, - lastBlockUC.getExpectedStorageLocations(), offset); + uc.getExpectedStorageLocations(), offset); return new FileState(file, src, iip); } else { // Case 3 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 008a32779d343..a61161ff171ea 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 @@ -42,15 +42,14 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.protocol.Block; 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.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.Storage; @@ -991,12 +990,14 @@ private void addNewBlock(AddBlockOp op, INodeFile file, final BlockInfo newBlockInfo; boolean isStriped = ecZone != null; if (isStriped) { - newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, + newBlockInfo = new BlockInfoStriped(newBlock, ecZone.getErasureCodingPolicy()); } else { - newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, + newBlockInfo = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); } + newBlockInfo.convertToBlockUnderConstruction( + BlockUCState.UNDER_CONSTRUCTION, null); fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file); file.addBlock(newBlockInfo); fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock); @@ -1077,12 +1078,14 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // what about an old-version fsync() where fsync isn't called // until several blocks in? if (isStriped) { - newBI = new BlockInfoStripedUnderConstruction(newBlock, + newBI = new BlockInfoStriped(newBlock, ecZone.getErasureCodingPolicy()); } else { - newBI = new BlockInfoContiguousUnderConstruction(newBlock, + newBI = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); } + newBI.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, + null); } else { // OP_CLOSE should add finalized blocks. This code path // is only executed when loading edits written by prior 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 e7c87d66ffea1..0b1902f2274a8 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 @@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; @@ -756,7 +755,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, // file // read blocks - Block[] blocks = new BlockInfoContiguous[numBlocks]; + BlockInfo[] blocks = new BlockInfoContiguous[numBlocks]; for (int j = 0; j < numBlocks; j++) { blocks[j] = new BlockInfoContiguous(replication); blocks[j].readFields(in); @@ -778,9 +777,9 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, clientMachine = FSImageSerialization.readString(in); // convert the last block to BlockUC if (blocks.length > 0) { - Block lastBlk = blocks[blocks.length - 1]; - blocks[blocks.length - 1] = - new BlockInfoContiguousUnderConstruction(lastBlk, replication); + BlockInfo lastBlk = blocks[blocks.length - 1]; + lastBlk.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); } } } 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 51b04d0bbf6c2..ffaf86b0c2077 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 @@ -45,10 +45,9 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary; @@ -378,11 +377,13 @@ private INodeFile loadINodeFile(INodeSection.INode n) { final BlockInfo ucBlk; if (isStriped) { BlockInfoStriped striped = (BlockInfoStriped) lastBlk; - ucBlk = new BlockInfoStripedUnderConstruction(striped, ecPolicy); + ucBlk = new BlockInfoStriped(striped, ecPolicy); } else { - ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, + ucBlk = new BlockInfoContiguous(lastBlk, replication); } + ucBlk.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); file.setBlock(file.numBlocks() - 1, ucBlk); } } 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 af3f8139d13d4..0567efd5f6a00 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 @@ -34,7 +34,6 @@ 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.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap; @@ -138,8 +137,9 @@ static INodeFile readINodeUnderConstruction( // last block is UNDER_CONSTRUCTION if(numBlocks > 0) { blk.readFields(in); - blocksContiguous[i] = new BlockInfoContiguousUnderConstruction( - blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); + blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication); + blocksContiguous[i].convertToBlockUnderConstruction( + BlockUCState.UNDER_CONSTRUCTION, null); } PermissionStatus perm = PermissionStatus.read(in); 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 a7107d7f37600..0d9d427bc004e 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 @@ -142,7 +142,6 @@ import org.apache.hadoop.fs.CacheFlag; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FsServerDefaults; @@ -204,10 +203,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; @@ -3124,28 +3122,25 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, throw new AlreadyBeingCreatedException(message); case UNDER_CONSTRUCTION: case UNDER_RECOVERY: - // TODO support truncate of striped blocks - final BlockInfoUnderConstruction uc = - (BlockInfoUnderConstruction)lastBlock; + final BlockUnderConstructionFeature uc = lastBlock.getUnderConstructionFeature(); // determine if last block was intended to be truncated Block recoveryBlock = uc.getTruncateBlock(); boolean truncateRecovery = recoveryBlock != null; boolean copyOnTruncate = truncateRecovery && - recoveryBlock.getBlockId() != uc.toBlock().getBlockId(); + recoveryBlock.getBlockId() != lastBlock.getBlockId(); assert !copyOnTruncate || - recoveryBlock.getBlockId() < uc.toBlock().getBlockId() && - recoveryBlock.getGenerationStamp() < uc.toBlock(). - getGenerationStamp() && - recoveryBlock.getNumBytes() > uc.toBlock().getNumBytes() : + recoveryBlock.getBlockId() < lastBlock.getBlockId() && + recoveryBlock.getGenerationStamp() < lastBlock.getGenerationStamp() && + recoveryBlock.getNumBytes() > lastBlock.getNumBytes() : "wrong recoveryBlock"; // setup the last block locations from the blockManager if not known if (uc.getNumExpectedLocations() == 0) { - uc.setExpectedLocations(blockManager.getStorages(lastBlock)); + uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock), + lastBlock.isStriped()); } - if (uc.getNumExpectedLocations() == 0 && - uc.toBlock().getNumBytes() == 0) { + if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) { // There is no datanode reported to this block. // may be client have crashed before writing data to pipeline. // This blocks doesn't need any recovery. @@ -3159,14 +3154,14 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, } // start recovery of the last block for this file long blockRecoveryId = - nextGenerationStamp(blockIdManager.isLegacyBlock(uc.toBlock())); + nextGenerationStamp(blockIdManager.isLegacyBlock(lastBlock)); lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile); if(copyOnTruncate) { - uc.toBlock().setGenerationStamp(blockRecoveryId); + lastBlock.setGenerationStamp(blockRecoveryId); } else if(truncateRecovery) { recoveryBlock.setGenerationStamp(blockRecoveryId); } - uc.initializeBlockRecovery(blockRecoveryId); + uc.initializeBlockRecovery(lastBlock, blockRecoveryId); leaseManager.renewLease(lease); // Cannot close file right now, since the last block requires recovery. // This may potentially cause infinite loop in lease recovery @@ -3371,8 +3366,8 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, } truncatedBlock = iFile.getLastBlock(); - final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)truncatedBlock; - final long recoveryId = uc.getBlockRecoveryId(); + final long recoveryId = truncatedBlock.getUnderConstructionFeature() + .getBlockRecoveryId(); copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId(); if(recoveryId != newgenerationstamp) { throw new IOException("The recovery id " + newgenerationstamp @@ -5424,7 +5419,7 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, // check the vadility of the block and lease holder name final INodeFile pendingFile = checkUCBlock(oldBlock, clientName); final BlockInfo lastBlock = pendingFile.getLastBlock(); - final BlockInfoUnderConstruction blockinfo = (BlockInfoUnderConstruction)lastBlock; + assert !lastBlock.isComplete(); // check new GS & length: this is not expected if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) { @@ -5444,12 +5439,13 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, // Update old block with the new generation stamp and new length lastBlock.setNumBytes(newBlock.getNumBytes()); - blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); + lastBlock.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); // find the DatanodeDescriptor objects final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager() .getDatanodeStorageInfos(newNodes, newStorageIDs); - blockinfo.setExpectedLocations(storages); + lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock, + storages, lastBlock.isStriped()); String src = pendingFile.getFullPathName(); FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache); 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 13f180aa85110..4fa457d033e9a 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 @@ -18,7 +18,6 @@ package org.apache.hadoop.hdfs.server.namenode; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; -import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION; import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID; import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID; @@ -39,12 +38,10 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList; import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature; @@ -86,7 +83,7 @@ public static INodeFile valueOf(INode inode, String path, boolean acceptNull) * [4-bit storagePolicyID][1-bit isStriped] * [11-bit replication][48-bit preferredBlockSize] */ - static enum HeaderFormat { + enum HeaderFormat { PREFERRED_BLOCK_SIZE(null, 48, 1), REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0), IS_STRIPED(REPLICATION.BITS, 1, 0), @@ -264,25 +261,20 @@ public void convertLastBlockToUC(BlockInfo lastBlock, if (numBlocks() == 0) { throw new IOException("Failed to set last block: File is empty."); } + lastBlock.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, + locations); + } - final BlockInfo ucBlock; - if (isStriped()) { - Preconditions.checkState(lastBlock.isStriped()); - ucBlock = ((BlockInfoStriped) lastBlock) - .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); - } else { - Preconditions.checkState(!lastBlock.isStriped()); - ucBlock = ((BlockInfoContiguous) lastBlock) - .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations); - } - setBlock(numBlocks() - 1, ucBlock); + void setLastBlock(BlockInfo blk) { + blk.setBlockCollection(this); + setBlock(numBlocks() - 1, blk); } /** * Remove a block from the block list. This block should be * the last one on the list. */ - BlockInfoUnderConstruction removeLastBlock(Block oldblock) { + BlockInfo removeLastBlock(Block oldblock) { Preconditions.checkState(isUnderConstruction(), "file is no longer under construction"); if (blocks == null || blocks.length == 0) { @@ -293,13 +285,12 @@ BlockInfoUnderConstruction removeLastBlock(Block oldblock) { return null; } - BlockInfoUnderConstruction uc = - (BlockInfoUnderConstruction)blocks[size_1]; + BlockInfo ucBlock = blocks[size_1]; //copy to a new list BlockInfo[] newlist = new BlockInfo[size_1]; System.arraycopy(blocks, 0, newlist, 0, size_1); setBlocks(newlist); - return uc; + return ucBlock; } /* End of Under-Construction Feature */ @@ -758,7 +749,7 @@ public final long computeFileSize(boolean includesLastUcBlock, //check if the last block is BlockInfoUnderConstruction BlockInfo lastBlk = blocks[last]; long size = lastBlk.getNumBytes(); - if (lastBlk instanceof BlockInfoUnderConstruction) { + if (!lastBlk.isComplete()) { if (!includesLastUcBlock) { size = 0; } else if (usePreferredBlockSize4LastUcBlock) { 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 442e127cdc52b..9dcd4d887f80f 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 @@ -23,7 +23,6 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; @@ -136,7 +135,7 @@ void combineAndCollectSnapshotBlocks( Block dontRemoveBlock = null; if (lastBlock != null && lastBlock.getBlockUCState().equals( HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) { - dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock) + dontRemoveBlock = lastBlock.getUnderConstructionFeature() .getTruncateBlock(); } // Collect the remaining blocks of the file, ignoring truncate block 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 7d895e31e02ae..6500b963871e0 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 @@ -117,7 +117,6 @@ 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.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; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -1650,13 +1649,11 @@ public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn, BlockManager bm0 = nn.getNamesystem().getBlockManager(); BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock()); assertTrue("Block " + blk + " should be under construction, " + - "got: " + storedBlock, - storedBlock instanceof BlockInfoContiguousUnderConstruction); - BlockInfoContiguousUnderConstruction ucBlock = - (BlockInfoContiguousUnderConstruction)storedBlock; + "got: " + storedBlock, !storedBlock.isComplete()); // We expect that the replica with the most recent heart beat will be // the one to be in charge of the synchronization / recovery protocol. - final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations(); + final DatanodeStorageInfo[] storages = storedBlock + .getUnderConstructionFeature().getExpectedStorageLocations(); DatanodeStorageInfo expectedPrimary = storages[0]; long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor() .getLastUpdateMonotonic(); 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 a447aafb9add8..c34747205dd55 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 @@ -39,25 +39,24 @@ public void testInitializeBlockRecovery() throws Exception { DatanodeDescriptor dd3 = s3.getDatanodeDescriptor(); dd1.isAlive = dd2.isAlive = dd3.isAlive = true; - BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( - new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), - (short) 3, - BlockUCState.UNDER_CONSTRUCTION, + BlockInfoContiguous blockInfo = new BlockInfoContiguous( + new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3); + blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, new DatanodeStorageInfo[] {s1, s2, s3}); // Recovery attempt #1. DFSTestUtil.resetLastUpdatesWithOffset(dd1, -3 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000); - blockInfo.initializeBlockRecovery(1); - BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1); + blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 1); + BlockInfo[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1); assertEquals(blockInfoRecovery[0], blockInfo); // Recovery attempt #2. DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000); - blockInfo.initializeBlockRecovery(2); + blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 2); blockInfoRecovery = dd1.getLeaseRecoveryCommand(1); assertEquals(blockInfoRecovery[0], blockInfo); @@ -65,7 +64,7 @@ public void testInitializeBlockRecovery() throws Exception { DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000); - blockInfo.initializeBlockRecovery(3); + blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3); blockInfoRecovery = dd3.getLeaseRecoveryCommand(1); assertEquals(blockInfoRecovery[0], blockInfo); @@ -74,7 +73,7 @@ public void testInitializeBlockRecovery() throws Exception { DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0); - blockInfo.initializeBlockRecovery(3); + blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3); blockInfoRecovery = dd3.getLeaseRecoveryCommand(1); assertEquals(blockInfoRecovery[0], blockInfo); } 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 66a4681c9384b..862f14d98dc44 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.blockmanagement; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -727,8 +728,8 @@ public void testSafeModeIBRBeforeFirstFullBR() throws Exception { // verify the storage info is correct assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo (ds) >= 0); - assertTrue(((BlockInfoContiguousUnderConstruction) bm. - getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0); + assertTrue(bm.getStoredBlock(new Block(receivingBlockId)) + .getUnderConstructionFeature().getNumExpectedLocations() > 0); assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId)) .findStorageInfo(ds) >= 0); assertNull(bm.getStoredBlock(new Block(ReceivedDeletedBlockId))); @@ -748,8 +749,8 @@ private BlockInfo addBlockToBM(long blkId) { private BlockInfo addUcBlockToBM(long blkId) { Block block = new Block(blkId); - BlockInfoContiguousUnderConstruction blockInfo = - new BlockInfoContiguousUnderConstruction(block, (short) 3); + BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3); + blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null); BlockCollection bc = Mockito.mock(BlockCollection.class); Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication(); bm.blocksMap.addBlockCollection(blockInfo, bc); 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 c1218a08392a0..3e233c67cc89b 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 @@ -172,9 +172,10 @@ public void testHeartbeatBlockRecovery() throws Exception { dd1.getStorageInfos()[0], dd2.getStorageInfos()[0], dd3.getStorageInfos()[0]}; - BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( - new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, - BlockUCState.UNDER_RECOVERY, storages); + BlockInfo blockInfo = new BlockInfoContiguous( + new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3); + blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY, + storages); dd1.addBlockToBeRecovered(blockInfo); DatanodeCommand[] cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); @@ -194,9 +195,10 @@ public void testHeartbeatBlockRecovery() throws Exception { // More than the default stale interval of 30 seconds. DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0); - blockInfo = new BlockInfoContiguousUnderConstruction( - new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, - BlockUCState.UNDER_RECOVERY, storages); + blockInfo = new BlockInfoContiguous( + new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3); + blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY, + storages); dd1.addBlockToBeRecovered(blockInfo); cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); assertEquals(1, cmds.length); @@ -215,9 +217,10 @@ public void testHeartbeatBlockRecovery() throws Exception { // More than the default stale interval of 30 seconds. DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000); - blockInfo = new BlockInfoContiguousUnderConstruction( - new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, - BlockUCState.UNDER_RECOVERY, storages); + blockInfo = new BlockInfoContiguous( + new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3); + blockInfo.convertToBlockUnderConstruction(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/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java index dcb852b62c4be..0172595bfd035 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 @@ -1182,7 +1182,8 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() // block under construction, the BlockManager will realize the expected // replication has been achieved and remove it from the under-replicated // queue. - BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1); + BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1); + info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null); BlockCollection bc = mock(BlockCollection.class); when(bc.getPreferredBlockReplication()).thenReturn((short)1); bm.addBlockCollection(info, bc); @@ -1247,9 +1248,8 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo( dataNodes[0], new DatanodeStorage("s1"))}; - final BlockInfoContiguousUnderConstruction ucBlock = - info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, - storageAry); + info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, + storageAry); DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class); DatanodeDescriptor dn = mock(DatanodeDescriptor.class); when(dn.isDecommissioned()).thenReturn(true); @@ -1258,10 +1258,10 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true); when(storage.addBlock(any(BlockInfo.class))).thenReturn (DatanodeStorageInfo.AddBlockResult.ADDED); - ucBlock.addStorage(storage, ucBlock); + info.addStorage(storage, info); BlockInfo lastBlk = mbc.getLastBlock(); - when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock); + when(mbc.getLastBlock()).thenReturn(lastBlk, info); bm.convertLastBlockToUnderConstruction(mbc, 0L); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index fa92c64f7888c..c472cd800d806 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -170,13 +169,13 @@ private void checkStripedBlockUC(BlockInfoStriped block, Assert.assertEquals(0, block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); - final BlockInfoStripedUnderConstruction blockUC = - (BlockInfoStripedUnderConstruction) block; Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, - blockUC.getBlockUCState()); + block.getBlockUCState()); if (checkReplica) { - Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations()); - DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations(); + Assert.assertEquals(GROUP_SIZE, + block.getUnderConstructionFeature().getNumExpectedLocations()); + DatanodeStorageInfo[] storages = block.getUnderConstructionFeature() + .getExpectedStorageLocations(); for (DataNode dn : cluster.getDataNodes()) { Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages)); } @@ -205,11 +204,10 @@ public void testGetLocatedStripedBlocks() throws Exception { FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); - BlockInfoStripedUnderConstruction lastBlk = - (BlockInfoStripedUnderConstruction) fileNode.getLastBlock(); - DatanodeInfo[] expectedDNs = DatanodeStorageInfo - .toDatanodeInfos(lastBlk.getExpectedStorageLocations()); - int[] indices = lastBlk.getBlockIndices(); + BlockInfoStriped lastBlk = (BlockInfoStriped) fileNode.getLastBlock(); + DatanodeInfo[] expectedDNs = DatanodeStorageInfo.toDatanodeInfos( + lastBlk.getUnderConstructionFeature().getExpectedStorageLocations()); + int[] indices = lastBlk.getUnderConstructionFeature().getBlockIndices(); LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L); Assert.assertEquals(1, blks.locatedBlockCount()); @@ -246,11 +244,10 @@ public void testAddUCReplica() throws Exception { cluster.getNamesystem().getAdditionalBlock(file.toString(), fileNode.getId(), dfs.getClient().getClientName(), null, null, null); BlockInfo lastBlock = fileNode.getLastBlock(); - BlockInfoStripedUnderConstruction ucBlock = - (BlockInfoStripedUnderConstruction) lastBlock; - DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); - int[] indices = ucBlock.getBlockIndices(); + DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature() + .getExpectedStorageLocations(); + int[] indices = lastBlock.getUnderConstructionFeature().getBlockIndices(); Assert.assertEquals(GROUP_SIZE, locs.length); Assert.assertEquals(GROUP_SIZE, indices.length); @@ -272,8 +269,8 @@ public void testAddUCReplica() throws Exception { } // make sure lastBlock is correct and the storages have been updated - locs = ucBlock.getExpectedStorageLocations(); - indices = ucBlock.getBlockIndices(); + locs = lastBlock.getUnderConstructionFeature().getExpectedStorageLocations(); + indices = lastBlock.getUnderConstructionFeature().getBlockIndices(); Assert.assertEquals(GROUP_SIZE, locs.length); Assert.assertEquals(GROUP_SIZE, indices.length); for (DatanodeStorageInfo newstorage : locs) { @@ -307,10 +304,9 @@ public void testAddUCReplica() throws Exception { bpId, reports, null); } - BlockInfoStripedUnderConstruction ucBlock = - (BlockInfoStripedUnderConstruction) lastBlock; - DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); - int[] indices = ucBlock.getBlockIndices(); + DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature() + .getExpectedStorageLocations(); + int[] indices = lastBlock.getUnderConstructionFeature().getBlockIndices(); Assert.assertEquals(GROUP_SIZE, locs.length); Assert.assertEquals(GROUP_SIZE, indices.length); for (i = 0; i < GROUP_SIZE; i++) { 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 f372bec3ba62e..ab6cde3066a11 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.namenode; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -36,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; 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.BlockUCState; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.junit.AfterClass; @@ -156,6 +157,7 @@ public void testBlockCreation() throws IOException { @Test public void testGetBlockLocations() throws IOException { final NamenodeProtocols namenode = cluster.getNameNodeRpc(); + final BlockManager blockManager = cluster.getNamesystem().getBlockManager(); final Path p = new Path(BASE_DIR, "file2.dat"); final String src = p.toString(); final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 3); @@ -170,7 +172,7 @@ public void testGetBlockLocations() throws IOException { final List blocks = lb.getLocatedBlocks(); assertEquals(i, blocks.size()); final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock(); - assertTrue(b instanceof BlockInfoContiguousUnderConstruction); + assertFalse(blockManager.getStoredBlock(b).isComplete()); if (++i < NUM_BLOCKS) { // write one more block 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 c218b7c2cc62d..35a098a38aaa2 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,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.junit.Test; @@ -68,11 +67,13 @@ private FSNamesystem makeNameSystemSpy(Block block, INodeFile file) namesystem.dir.getINodeMap().put(file); FSNamesystem namesystemSpy = spy(namesystem); - BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( - block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); + BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 1); + blockInfo.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); blockInfo.setBlockCollection(file); blockInfo.setGenerationStamp(genStamp); - blockInfo.initializeBlockRecovery(genStamp); + blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, + genStamp); doReturn(blockInfo).when(file).removeLastBlock(any(Block.class)); doReturn(true).when(file).isUnderConstruction(); doReturn(new BlockInfoContiguous[1]).when(file).getBlocks(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java index f6aae22779f2a..74368792cf52a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java @@ -55,7 +55,6 @@ 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.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.security.UserGroupInformation; @@ -1017,7 +1016,7 @@ public void testTruncateRecovery() throws IOException { is(fsn.getBlockIdManager().getGenerationStampV2())); assertThat(file.getLastBlock().getBlockUCState(), is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY)); - long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock()) + long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature() .getBlockRecoveryId(); assertThat(blockRecoveryId, is(initialGenStamp + 1)); fsn.getEditLog().logTruncate( @@ -1051,7 +1050,7 @@ public void testTruncateRecovery() throws IOException { is(fsn.getBlockIdManager().getGenerationStampV2())); assertThat(file.getLastBlock().getBlockUCState(), is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY)); - long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock()) + long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature() .getBlockRecoveryId(); assertThat(blockRecoveryId, is(initialGenStamp + 1)); fsn.getEditLog().logTruncate( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 8b1a11fdd1bc8..48b22c064d750 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -157,8 +156,9 @@ public void testBlockStripedUCFileSize() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); - BlockInfoStripedUnderConstruction bInfoUCStriped - = new BlockInfoStripedUnderConstruction(blk, testECPolicy); + BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy); + bInfoUCStriped.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); bInfoUCStriped.setNumBytes(100); inf.addBlock(bInfoUCStriped); assertEquals(100, inf.computeFileSize()); @@ -191,8 +191,9 @@ public void testBlockStripedUCComputeQuotaUsage() throws IOException, InterruptedException { INodeFile inf = createStripedINodeFile(); Block blk = new Block(1); - BlockInfoStripedUnderConstruction bInfoUCStriped - = new BlockInfoStripedUnderConstruction(blk, testECPolicy); + BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy); + bInfoUCStriped.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); bInfoUCStriped.setNumBytes(100); inf.addBlock(bInfoUCStriped); 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 d202fb788f8f0..c27ead5f659e8 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 @@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper; @@ -752,12 +752,13 @@ void invoke() throws Exception { boolean checkNamenodeBeforeReturn() throws Exception { INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory() .getINode4Write(file).asFile(); - BlockInfoContiguousUnderConstruction blkUC = - (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1]; - int datanodeNum = blkUC.getExpectedStorageLocations().length; + BlockInfo blkUC = (fileNode.getBlocks())[1]; + int datanodeNum = blkUC.getUnderConstructionFeature() + .getExpectedStorageLocations().length; for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) { Thread.sleep(1000); - datanodeNum = blkUC.getExpectedStorageLocations().length; + datanodeNum = blkUC.getUnderConstructionFeature() + .getExpectedStorageLocations().length; } return datanodeNum == 2; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java index 128bd9edbcc83..9a611aa0b7ff9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java @@ -43,10 +43,9 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature; import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage; import org.apache.hadoop.hdfs.server.datanode.BlockScanner; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -178,8 +177,7 @@ public static void checkSnapshotCreation(DistributedFileSystem hdfs, * Specific information for different types of INode: * {@link INodeDirectory}:childrenSize * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()} - * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information. - * {@link FileWithSnapshot}: next link + * and {@link BlockUnderConstructionFeature#toString()} for detailed information. * * @see INode#dumpTreeRecursively() */ From ddf4e785475affead2f7c070b9f151de0fcb9024 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 2 Sep 2015 11:45:45 -0700 Subject: [PATCH 197/212] HDFS-8937. Erasure coding: do not throw exception when setting replication factor on EC files. Contributed by Gao Rui. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hadoop/hdfs/server/namenode/FSDirAttrOp.java | 9 +++------ .../org/apache/hadoop/hdfs/TestErasureCodingZones.java | 10 ++-------- 3 files changed, 8 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 28cc34aa922ad..fb464bf2af416 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -406,3 +406,6 @@ HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549) + + HDFS-8937. Erasure coding: do not throw exception when setting replication on + EC file. (Gao Rui via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index 4bed13ebc8a8f..46e172d3d69a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -405,15 +405,12 @@ static BlockInfo[] unprotectedSetReplication( final BlockManager bm = fsd.getBlockManager(); final INodesInPath iip = fsd.getINodesInPath4Write(src, true); final INode inode = iip.getLastINode(); - if (inode == null || !inode.isFile()) { + if (inode == null || !inode.isFile() || inode.asFile().isStriped()) { + // TODO we do not support replication on stripe layout files yet return null; } - INodeFile file = inode.asFile(); - if (file.isStriped()) { - throw new UnsupportedActionException( - "Cannot set replication to a file with striped blocks"); - } + INodeFile file = inode.asFile(); // Make sure the directory has sufficient quotas short oldBR = file.getPreferredBlockReplication(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java index a878501f5b008..b68aab99563b7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java @@ -166,14 +166,8 @@ public void testReplication() throws IOException { fs.create(fooFile, FsPermission.getFileDefault(), true, conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), (short)0, fs.getDefaultBlockSize(fooFile), null); - - try { - fs.setReplication(fooFile, (short) 3); - fail("Shouldn't allow to set replication to a file with striped blocks"); - } catch (IOException e) { - assertExceptionContains( - "Cannot set replication to a file with striped blocks", e); - } + // set replication should be a no-op + fs.setReplication(fooFile, (short) 3); } @Test From 60bd765ac1b7d21189efc47058c00ff956a2dc86 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Wed, 2 Sep 2015 17:41:08 -0700 Subject: [PATCH 198/212] HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream. Contributed by Walter Su. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../hadoop/hdfs/StripedDataStreamer.java | 3 +- .../hadoop/hdfs/StripedFileTestUtil.java | 57 +++++++++++++++++++ .../hdfs/TestDFSStripedInputStream.java | 1 + .../hdfs/TestDFSStripedOutputStream.java | 32 ++++++----- ...TestDFSStripedOutputStreamWithFailure.java | 9 +++ .../hdfs/TestReadStripedFileWithDecoding.java | 9 ++- .../TestReadStripedFileWithMissingBlocks.java | 18 +++--- .../hadoop/hdfs/TestRecoverStripedFile.java | 26 ++------- .../hadoop/hdfs/TestWriteReadStripedFile.java | 35 ++++++------ 10 files changed, 130 insertions(+), 63 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index fb464bf2af416..9fdf3ed949de8 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -409,3 +409,6 @@ HDFS-8937. Erasure coding: do not throw exception when setting replication on EC file. (Gao Rui via jing9) + + HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream. + (Walter Su via jing9) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index a20caa53326f1..a2b6a676d4ef0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -67,7 +67,7 @@ static abstract class ConcurrentPoll { this.queue = queue; } - T poll(final int i) throws InterruptedIOException { + T poll(final int i) throws IOException { for(;;) { synchronized(queue) { final T polled = queue.poll(i); @@ -80,6 +80,7 @@ T poll(final int i) throws InterruptedIOException { return queue.poll(i); } catch(IOException ioe) { LOG.warn("Failed to populate, " + this, ioe); + throw ioe; } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index ca4b2aabd3a20..ded86e7fb384b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs; +import com.google.common.base.Joiner; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; @@ -25,6 +26,8 @@ import org.apache.hadoop.fs.Path; 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.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.web.ByteRangeInputStream; import org.junit.Assert; @@ -32,8 +35,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Random; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + public class StripedFileTestUtil { public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class); @@ -224,6 +230,57 @@ static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { } } + /** + * If the length of blockGroup is less than a full stripe, it returns the the + * number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS. + */ + public static short getRealDataBlockNum(int numBytes) { + return (short) Math.min(dataBlocks, + (numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1); + } + + public static short getRealTotalBlockNum(int numBytes) { + return (short) (getRealDataBlockNum(numBytes) + parityBlocks); + } + + /** + * Wait for all the internalBlocks of the blockGroups of the given file to be reported. + */ + public static void waitBlockGroupsReported(DistributedFileSystem fs, String src) + throws IOException, InterruptedException, TimeoutException { + boolean success; + final int ATTEMPTS = 40; + int count = 0; + + do { + success = true; + count++; + LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0); + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + short expected = getRealTotalBlockNum((int) lb.getBlockSize()); + int reported = lb.getLocations().length; + if (reported != expected){ + success = false; + System.out.println("blockGroup " + lb.getBlock() + " of file " + src + + " has reported internalBlocks " + reported + + " (desired " + expected + "); locations " + + Joiner.on(' ').join(lb.getLocations())); + Thread.sleep(1000); + break; + } + } + if (success) { + System.out.println("All blockGroups of file " + src + + " verified to have all internalBlocks."); + } + } while (!success && count < ATTEMPTS); + + if (count == ATTEMPTS) { + throw new TimeoutException("Timed out waiting for " + src + + " to have all the internalBlocks"); + } + } + /** * Generate n random and different numbers within * specified non-negative integer range diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index ed3c11000dccf..a09f0f0ccfa9e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -70,6 +70,7 @@ public class TestDFSStripedInputStream { @Before public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); SimulatedFSDataset.setFactory(conf); cluster = new MiniDFSCluster.Builder(conf).numDataNodes( DATA_BLK_NUM + PARITY_BLK_NUM).build(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 35e7e6d3b9661..a467f40d8b6d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -66,6 +66,7 @@ public void setup() throws IOException { int numDNs = dataBlocks + parityBlocks + 2; conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); @@ -79,76 +80,76 @@ public void tearDown() { } @Test - public void testFileEmpty() throws IOException { + public void testFileEmpty() throws Exception { testOneFile("/EmptyFile", 0); } @Test - public void testFileSmallerThanOneCell1() throws IOException { + public void testFileSmallerThanOneCell1() throws Exception { testOneFile("/SmallerThanOneCell", 1); } @Test - public void testFileSmallerThanOneCell2() throws IOException { + public void testFileSmallerThanOneCell2() throws Exception { testOneFile("/SmallerThanOneCell", cellSize - 1); } @Test - public void testFileEqualsWithOneCell() throws IOException { + public void testFileEqualsWithOneCell() throws Exception { testOneFile("/EqualsWithOneCell", cellSize); } @Test - public void testFileSmallerThanOneStripe1() throws IOException { + public void testFileSmallerThanOneStripe1() throws Exception { testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1); } @Test - public void testFileSmallerThanOneStripe2() throws IOException { + public void testFileSmallerThanOneStripe2() throws Exception { testOneFile("/SmallerThanOneStripe", cellSize + 123); } @Test - public void testFileEqualsWithOneStripe() throws IOException { + public void testFileEqualsWithOneStripe() throws Exception { testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks); } @Test - public void testFileMoreThanOneStripe1() throws IOException { + public void testFileMoreThanOneStripe1() throws Exception { testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123); } @Test - public void testFileMoreThanOneStripe2() throws IOException { + public void testFileMoreThanOneStripe2() throws Exception { testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks + cellSize * dataBlocks + 123); } @Test - public void testFileLessThanFullBlockGroup() throws IOException { + public void testFileLessThanFullBlockGroup() throws Exception { testOneFile("/LessThanFullBlockGroup", cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); } @Test - public void testFileFullBlockGroup() throws IOException { + public void testFileFullBlockGroup() throws Exception { testOneFile("/FullBlockGroup", blockSize * dataBlocks); } @Test - public void testFileMoreThanABlockGroup1() throws IOException { + public void testFileMoreThanABlockGroup1() throws Exception { testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); } @Test - public void testFileMoreThanABlockGroup2() throws IOException { + public void testFileMoreThanABlockGroup2() throws Exception { testOneFile("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123); } @Test - public void testFileMoreThanABlockGroup3() throws IOException { + public void testFileMoreThanABlockGroup3() throws Exception { testOneFile("/MoreThanABlockGroup3", blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123); @@ -167,12 +168,13 @@ private byte getByte(long pos) { return (byte) (pos % mod + 1); } - private void testOneFile(String src, int writeBytes) throws IOException { + private void testOneFile(String src, int writeBytes) throws Exception { src += "_" + writeBytes; Path testPath = new Path(src); byte[] bytes = generateBytes(writeBytes); DFSTestUtil.writeFile(fs, testPath, new String(bytes)); + StripedFileTestUtil.waitBlockGroupsReported(fs, src); // check file length FileStatus status = fs.getFileStatus(testPath); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index e8e562b53adea..2aa8484419bef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -56,6 +56,7 @@ import com.google.common.base.Preconditions; + public class TestDFSStripedOutputStreamWithFailure { public static final Log LOG = LogFactory.getLog( TestDFSStripedOutputStreamWithFailure.class); @@ -135,6 +136,7 @@ private HdfsConfiguration newHdfsConfiguration() { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); return conf; } @@ -331,6 +333,13 @@ private void runTest(final int length, final int killPos, } } out.close(); + + short expectedReported = StripedFileTestUtil.getRealTotalBlockNum(length); + if (length > dnIndex * CELL_SIZE || dnIndex >= NUM_DATA_BLOCKS) { + expectedReported--; + } + DFSTestUtil.waitReplication(dfs, p, expectedReported); + Assert.assertTrue(killed); // check file length diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index cb2ec118e16eb..9048fbdfddb35 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -19,6 +19,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; @@ -62,6 +63,9 @@ public class TestReadStripedFileWithDecoding { @Before public void setup() throws IOException { + Configuration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()) .numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); @@ -80,7 +84,7 @@ public void tearDown() throws IOException { * Verify the decoding works correctly. */ @Test(timeout=300000) - public void testReadWithDNFailure() throws IOException { + public void testReadWithDNFailure() throws Exception { for (int fileLength : fileLengths) { for (int dnFailureNum : dnFailureNums) { try { @@ -161,7 +165,7 @@ private void verifyRead(Path testPath, int length, byte[] expected) } private void testReadWithDNFailure(int fileLength, int dnFailureNum) - throws IOException { + throws Exception { String fileType = fileLength < (blockSize * dataBlocks) ? "smallFile" : "largeFile"; String src = "/dnFailure_" + dnFailureNum + "_" + fileType; @@ -172,6 +176,7 @@ private void testReadWithDNFailure(int fileLength, int dnFailureNum) Path testPath = new Path(src); final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength); DFSTestUtil.writeFile(fs, testPath, bytes); + StripedFileTestUtil.waitBlockGroupsReported(fs, src); // shut down the DN that holds an internal data block BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java index 38256ab256f18..08a5f58591091 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java @@ -44,13 +44,14 @@ public class TestReadStripedFileWithMissingBlocks { public static final Log LOG = LogFactory .getLog(TestReadStripedFileWithMissingBlocks.class); private static MiniDFSCluster cluster; - private static FileSystem fs; + private static DistributedFileSystem fs; private static Configuration conf = new HdfsConfiguration(); private final int fileLength = blockSize * dataBlocks + 123; @Before public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().createErasureCodingZone("/", null); fs = cluster.getFileSystem(); @@ -64,42 +65,43 @@ public void tearDown() throws IOException { } @Test - public void testReadFileWithMissingBlocks1() throws IOException { + public void testReadFileWithMissingBlocks1() throws Exception { readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 0); } @Test - public void testReadFileWithMissingBlocks2() throws IOException { + public void testReadFileWithMissingBlocks2() throws Exception { readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 1); } @Test - public void testReadFileWithMissingBlocks3() throws IOException { + public void testReadFileWithMissingBlocks3() throws Exception { readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 2); } @Test - public void testReadFileWithMissingBlocks4() throws IOException { + public void testReadFileWithMissingBlocks4() throws Exception { readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 0); } @Test - public void testReadFileWithMissingBlocks5() throws IOException { + public void testReadFileWithMissingBlocks5() throws Exception { readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 1); } @Test - public void testReadFileWithMissingBlocks6() throws IOException { + public void testReadFileWithMissingBlocks6() throws Exception { readFileWithMissingBlocks(new Path("/foo"), fileLength, 3, 0); } private void readFileWithMissingBlocks(Path srcPath, int fileLength, int missingDataNum, int missingParityNum) - throws IOException { + throws Exception { LOG.info("readFileWithMissingBlocks: (" + missingDataNum + "," + missingParityNum + ")"); final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + StripedFileTestUtil.waitBlockGroupsReported(fs, srcPath.toUri().getPath()); StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); int dataBlocks = (fileLength - 1) / cellSize + 1; BlockLocation[] locs = fs.getFileBlockLocations(srcPath, 0, cellSize); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index 5c173592eeb1a..7a0851f462022 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -182,9 +182,12 @@ private void assertFileBlocksRecovery(String fileName, int fileLen, } Path file = new Path(fileName); - - testCreateStripedFile(file, fileLen); - + + final byte[] data = new byte[fileLen]; + ThreadLocalRandom.current().nextBytes(data); + DFSTestUtil.writeFile(fs, file, data); + StripedFileTestUtil.waitBlockGroupsReported(fs, fileName); + LocatedBlocks locatedBlocks = getLocatedBlocks(file); assertEquals(locatedBlocks.getFileLength(), fileLen); @@ -380,21 +383,4 @@ private LocatedBlocks waitForRecoveryFinished(Path file, int groupSize) private LocatedBlocks getLocatedBlocks(Path file) throws IOException { return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE); } - - private void testCreateStripedFile(Path file, int dataLen) - throws IOException { - final byte[] data = new byte[dataLen]; - ThreadLocalRandom.current().nextBytes(data); - writeContents(file, data); - } - - void writeContents(Path file, byte[] contents) - throws IOException { - FSDataOutputStream out = fs.create(file); - try { - out.write(contents, 0, contents.length); - } finally { - out.close(); - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 810edb291d2be..b383c85889c6b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -45,7 +45,7 @@ public class TestWriteReadStripedFile { public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class); private static MiniDFSCluster cluster; - private static FileSystem fs; + private static DistributedFileSystem fs; private static Configuration conf = new HdfsConfiguration(); static { @@ -69,32 +69,32 @@ public void tearDown() throws IOException { } @Test - public void testFileEmpty() throws IOException { + public void testFileEmpty() throws Exception { testOneFileUsingDFSStripedInputStream("/EmptyFile", 0); testOneFileUsingDFSStripedInputStream("/EmptyFile2", 0, true); } @Test - public void testFileSmallerThanOneCell1() throws IOException { + public void testFileSmallerThanOneCell1() throws Exception { testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1); testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", 1, true); } @Test - public void testFileSmallerThanOneCell2() throws IOException { + public void testFileSmallerThanOneCell2() throws Exception { testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1); testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", cellSize - 1, true); } @Test - public void testFileEqualsWithOneCell() throws IOException { + public void testFileEqualsWithOneCell() throws Exception { testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize); testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell2", cellSize, true); } @Test - public void testFileSmallerThanOneStripe1() throws IOException { + public void testFileSmallerThanOneStripe1() throws Exception { testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize * dataBlocks - 1); testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2", @@ -102,7 +102,7 @@ public void testFileSmallerThanOneStripe1() throws IOException { } @Test - public void testFileSmallerThanOneStripe2() throws IOException { + public void testFileSmallerThanOneStripe2() throws Exception { testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize + 123); testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2", @@ -110,7 +110,7 @@ public void testFileSmallerThanOneStripe2() throws IOException { } @Test - public void testFileEqualsWithOneStripe() throws IOException { + public void testFileEqualsWithOneStripe() throws Exception { testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", cellSize * dataBlocks); testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe2", @@ -118,7 +118,7 @@ public void testFileEqualsWithOneStripe() throws IOException { } @Test - public void testFileMoreThanOneStripe1() throws IOException { + public void testFileMoreThanOneStripe1() throws Exception { testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", cellSize * dataBlocks + 123); testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe12", @@ -126,7 +126,7 @@ public void testFileMoreThanOneStripe1() throws IOException { } @Test - public void testFileMoreThanOneStripe2() throws IOException { + public void testFileMoreThanOneStripe2() throws Exception { testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", cellSize * dataBlocks + cellSize * dataBlocks + 123); testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe22", @@ -134,7 +134,7 @@ public void testFileMoreThanOneStripe2() throws IOException { } @Test - public void testLessThanFullBlockGroup() throws IOException { + public void testLessThanFullBlockGroup() throws Exception { testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup", cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup2", @@ -142,7 +142,7 @@ public void testLessThanFullBlockGroup() throws IOException { } @Test - public void testFileFullBlockGroup() throws IOException { + public void testFileFullBlockGroup() throws Exception { testOneFileUsingDFSStripedInputStream("/FullBlockGroup", blockSize * dataBlocks); testOneFileUsingDFSStripedInputStream("/FullBlockGroup2", @@ -150,7 +150,7 @@ public void testFileFullBlockGroup() throws IOException { } @Test - public void testFileMoreThanABlockGroup1() throws IOException { + public void testFileMoreThanABlockGroup1() throws Exception { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup12", @@ -158,7 +158,7 @@ public void testFileMoreThanABlockGroup1() throws IOException { } @Test - public void testFileMoreThanABlockGroup2() throws IOException { + public void testFileMoreThanABlockGroup2() throws Exception { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize + 123); testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup22", @@ -167,7 +167,7 @@ public void testFileMoreThanABlockGroup2() throws IOException { @Test - public void testFileMoreThanABlockGroup3() throws IOException { + public void testFileMoreThanABlockGroup3() throws Exception { testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3", blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123); @@ -177,15 +177,16 @@ public void testFileMoreThanABlockGroup3() throws IOException { } private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) - throws IOException { + throws Exception { testOneFileUsingDFSStripedInputStream(src, fileLength, false); } private void testOneFileUsingDFSStripedInputStream(String src, int fileLength, - boolean withDataNodeFailure) throws IOException { + boolean withDataNodeFailure) throws Exception { final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); Path srcPath = new Path(src); DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + StripedFileTestUtil.waitBlockGroupsReported(fs, src); StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); From f62237bc2f02afe11ce185e13aa51a60b5960037 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 9 Sep 2015 23:07:20 -0700 Subject: [PATCH 199/212] HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and eliminate notion of EC zones. --- .../hadoop/hdfs/protocol/ClientProtocol.java | 14 +- .../hdfs/protocol/ErasureCodingZone.java | 54 ------ .../main/proto/ClientNamenodeProtocol.proto | 8 +- .../src/main/proto/erasurecoding.proto | 20 +-- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../org/apache/hadoop/hdfs/DFSClient.java | 22 +-- .../hadoop/hdfs/DistributedFileSystem.java | 40 ++--- .../apache/hadoop/hdfs/client/HdfsAdmin.java | 23 +-- .../hdfs/protocol/ErasureCodingZone.java | 54 ------ ...amenodeProtocolServerSideTranslatorPB.java | 29 ++-- .../ClientNamenodeProtocolTranslatorPB.java | 27 ++- .../hadoop/hdfs/protocolPB/PBHelper.java | 15 -- .../server/blockmanagement/BlockManager.java | 16 +- .../server/common/HdfsServerConstants.java | 4 +- .../namenode/ErasureCodingPolicyManager.java | 2 +- .../namenode/ErasureCodingZoneManager.java | 163 ------------------ .../server/namenode/FSDirErasureCodingOp.java | 159 +++++++++++------ .../hdfs/server/namenode/FSDirRenameOp.java | 2 - .../namenode/FSDirStatAndListingOp.java | 17 +- .../server/namenode/FSDirWriteFileOp.java | 10 +- .../hdfs/server/namenode/FSDirectory.java | 4 - .../hdfs/server/namenode/FSEditLogLoader.java | 32 ++-- .../hdfs/server/namenode/FSNamesystem.java | 39 ++--- .../hdfs/server/namenode/INodeFile.java | 14 ++ .../server/namenode/INodeFileAttributes.java | 10 +- .../server/namenode/NameNodeRpcServer.java | 15 +- .../hdfs/server/namenode/Namesystem.java | 10 +- .../hdfs/tools/erasurecode/ECCommand.java | 42 ++--- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 4 +- .../hdfs/TestDFSStripedInputStream.java | 2 +- .../hdfs/TestDFSStripedOutputStream.java | 2 +- ...TestDFSStripedOutputStreamWithFailure.java | 2 +- ...es.java => TestErasureCodingPolicies.java} | 106 ++++++------ .../hdfs/TestFileStatusWithECPolicy.java | 10 +- .../hdfs/TestReadStripedFileWithDecoding.java | 2 +- .../TestReadStripedFileWithMissingBlocks.java | 2 +- .../hadoop/hdfs/TestRecoverStripedFile.java | 2 +- .../hdfs/TestSafeModeWithStripedFile.java | 2 +- .../hadoop/hdfs/TestWriteReadStripedFile.java | 2 +- .../hdfs/TestWriteStripedFileWithFailure.java | 5 +- .../hdfs/server/balancer/TestBalancer.java | 3 +- .../TestBlockTokenWithDFSStriped.java | 2 +- .../TestSequentialBlockGroupId.java | 14 +- .../hadoop/hdfs/server/mover/TestMover.java | 4 +- .../TestAddOverReplicatedStripedBlocks.java | 4 +- .../server/namenode/TestAddStripedBlocks.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 4 +- .../hdfs/server/namenode/TestFSImage.java | 4 +- .../namenode/TestQuotaWithStripedBlocks.java | 3 +- .../server/namenode/TestStripedINodeFile.java | 20 +-- ...stOfflineImageViewerWithStripedBlocks.java | 2 +- .../test/resources/testErasureCodingConf.xml | 150 ++++++++-------- 52 files changed, 474 insertions(+), 727 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java rename hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/{TestErasureCodingZones.java => TestErasureCodingPolicies.java} (67%) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java index 7756bb94f25df..a0504a8fba39e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java @@ -1485,11 +1485,13 @@ List listXAttrs(String src) EventBatchList getEditsFromTxid(long txid) throws IOException; /** - * Create an erasure coding zone with specified policy, if any, otherwise - * default + * Set an erasure coding policy on a specified path. + * @param src The path to set policy on. + * @param ecPolicy The erasure coding policy. If null, default policy will + * be used */ @AtMostOnce - void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) + void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) throws IOException; /** @@ -1501,11 +1503,11 @@ void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException; /** - * Get the information about the EC zone for the path - * + * Get the information about the EC policy for the path + * * @param src path to get the info for * @throws IOException */ @Idempotent - ErasureCodingZone getErasureCodingZone(String src) throws IOException; + ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java deleted file mode 100644 index 533b630893c15..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.hdfs.protocol; - -/** - * Information about the EC Zone at the specified path. - */ -public class ErasureCodingZone { - - private String dir; - private ErasureCodingPolicy ecPolicy; - - public ErasureCodingZone(String dir, ErasureCodingPolicy ecPolicy) { - this.dir = dir; - this.ecPolicy = ecPolicy; - } - - /** - * Get directory of the EC zone. - * - * @return - */ - public String getDir() { - return dir; - } - - /** - * Get the erasure coding policy for the EC Zone - * - * @return - */ - public ErasureCodingPolicy getErasureCodingPolicy() { - return ecPolicy; - } - - @Override - public String toString() { - return "Dir: " + getDir() + ", Policy: " + ecPolicy; - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto index fb10e9c7d0441..6a140ebe7ef59 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto @@ -867,14 +867,14 @@ service ClientNamenodeProtocol { returns(ListEncryptionZonesResponseProto); rpc getEZForPath(GetEZForPathRequestProto) returns(GetEZForPathResponseProto); - rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto) - returns(CreateErasureCodingZoneResponseProto); + rpc setErasureCodingPolicy(SetErasureCodingPolicyRequestProto) + returns(SetErasureCodingPolicyResponseProto); rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto) returns(GetCurrentEditLogTxidResponseProto); rpc getEditsFromTxid(GetEditsFromTxidRequestProto) returns(GetEditsFromTxidResponseProto); rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto) returns(GetErasureCodingPoliciesResponseProto); - rpc getErasureCodingZone(GetErasureCodingZoneRequestProto) - returns(GetErasureCodingZoneResponseProto); + rpc getErasureCodingPolicy(GetErasureCodingPolicyRequestProto) + returns(GetErasureCodingPolicyResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto index d27f7828f4882..fa24aefa11dae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto @@ -23,20 +23,12 @@ package hadoop.hdfs; import "hdfs.proto"; -/** - * ErasureCodingZone - */ -message ErasureCodingZoneProto { - required string dir = 1; - required ErasureCodingPolicyProto ecPolicy = 2; -} - -message CreateErasureCodingZoneRequestProto { +message SetErasureCodingPolicyRequestProto { required string src = 1; optional ErasureCodingPolicyProto ecPolicy = 2; } -message CreateErasureCodingZoneResponseProto { +message SetErasureCodingPolicyResponseProto { } message GetErasureCodingPoliciesRequestProto { // void request @@ -46,12 +38,12 @@ message GetErasureCodingPoliciesResponseProto { repeated ErasureCodingPolicyProto ecPolicies = 1; } -message GetErasureCodingZoneRequestProto { - required string src = 1; // path to get the zone info +message GetErasureCodingPolicyRequestProto { + required string src = 1; // path to get the policy info } -message GetErasureCodingZoneResponseProto { - optional ErasureCodingZoneProto ECZone = 1; +message GetErasureCodingPolicyResponseProto { + optional ErasureCodingPolicyProto ecPolicy = 1; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 9fdf3ed949de8..2f133103e635a 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -412,3 +412,6 @@ HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream. (Walter Su via jing9) + + HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and + eliminate notion of EC zones. (zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 12120f4b9287e..4d28dca1015bc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -114,7 +114,6 @@ 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.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -2901,12 +2900,13 @@ public RemoteIterator listEncryptionZones() return new EncryptionZoneIterator(namenode, traceSampler); } - public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) + + public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) throws IOException { checkOpen(); - TraceScope scope = getPathTraceScope("createErasureCodingZone", src); + TraceScope scope = getPathTraceScope("setErasureCodingPolicy", src); try { - namenode.createErasureCodingZone(src, ecPolicy); + namenode.setErasureCodingPolicy(src, ecPolicy); } catch (RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, SafeModeException.class, @@ -3223,17 +3223,19 @@ TraceScope getSrcDstTraceScope(String description, String src, String dst) { } /** - * Get the erasure coding zone information for the specified path - * + * Get the erasure coding policy information for the specified path + * * @param src path to get the information for - * @return Returns the zone information if path is in EC Zone, null otherwise + * @return Returns the policy information if file or directory on the path is + * erasure coded, null otherwise * @throws IOException */ - public ErasureCodingZone getErasureCodingZone(String src) throws IOException { + + public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException { checkOpen(); - TraceScope scope = getPathTraceScope("getErasureCodingZone", src); + TraceScope scope = getPathTraceScope("getErasureCodingPolicy", src); try { - return namenode.getErasureCodingZone(src); + return namenode.getErasureCodingPolicy(src); } catch (RemoteException re) { throw re.unwrapRemoteException(FileNotFoundException.class, AccessControlException.class, UnresolvedPathException.class); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 1f8ab4486970d..903f763d720b8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -73,7 +73,6 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -2263,20 +2262,20 @@ public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid) } /** - * Create the erasurecoding zone - * - * @param path Directory to create the ec zone - * @param ecPolicy erasure coding policy for the zone. If not specified default will be used. + * Set the source path to the specified erasure coding policy. + * + * @param path The directory to set the policy + * @param ecPolicy The erasure coding policy. If not specified default will be used. * @throws IOException */ - public void createErasureCodingZone(final Path path, final ErasureCodingPolicy ecPolicy) + public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy) throws IOException { Path absF = fixRelativePart(path); new FileSystemLinkResolver() { @Override public Void doCall(final Path p) throws IOException, UnresolvedLinkException { - dfs.createErasureCodingZone(getPathName(p), ecPolicy); + dfs.setErasureCodingPolicy(getPathName(p), ecPolicy); return null; } @@ -2284,42 +2283,43 @@ public Void doCall(final Path p) throws IOException, public Void next(final FileSystem fs, final Path p) throws IOException { if (fs instanceof DistributedFileSystem) { DistributedFileSystem myDfs = (DistributedFileSystem) fs; - myDfs.createErasureCodingZone(p, ecPolicy); + myDfs.setErasureCodingPolicy(p, ecPolicy); return null; } throw new UnsupportedOperationException( - "Cannot createErasureCodingZone through a symlink to a " + "Cannot setErasureCodingPolicy through a symlink to a " + "non-DistributedFileSystem: " + path + " -> " + p); } }.resolve(this, absF); } /** - * Get ErasureCoding zone information for the specified path - * - * @param path - * @return Returns the zone information if path is in EC zone, null otherwise + * Get erasure coding policy information for the specified path + * + * @param path The path of the file or directory + * @return Returns the policy information if file or directory on the path + * is erasure coded, null otherwise * @throws IOException */ - public ErasureCodingZone getErasureCodingZone(final Path path) + public ErasureCodingPolicy getErasureCodingPolicy(final Path path) throws IOException { Path absF = fixRelativePart(path); - return new FileSystemLinkResolver() { + return new FileSystemLinkResolver() { @Override - public ErasureCodingZone doCall(final Path p) throws IOException, + public ErasureCodingPolicy doCall(final Path p) throws IOException, UnresolvedLinkException { - return dfs.getErasureCodingZone(getPathName(p)); + return dfs.getErasureCodingPolicy(getPathName(p)); } @Override - public ErasureCodingZone next(final FileSystem fs, final Path p) + public ErasureCodingPolicy next(final FileSystem fs, final Path p) throws IOException { if (fs instanceof DistributedFileSystem) { DistributedFileSystem myDfs = (DistributedFileSystem) fs; - return myDfs.getErasureCodingZone(p); + return myDfs.getErasureCodingPolicy(p); } throw new UnsupportedOperationException( - "Cannot getErasureCodingZone through a symlink to a " + "Cannot getErasureCodingPolicy through a symlink to a " + "non-DistributedFileSystem: " + path + " -> " + p); } }.resolve(this, absF); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java index e6e67cbbe2c17..7b055fc8f6458 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry; import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.EncryptionZone; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.hdfs.tools.DFSAdmin; @@ -367,31 +366,33 @@ public void setStoragePolicy(final Path src, final String policyName) } /** - * Create the ErasureCoding zone + * Set the source path to the specified erasure coding policy. * - * @param path Directory to create the ErasureCoding zone - * @param ecPolicy erasure coding policy for the zone. If null, the default will be used. + * @param path The source path referring to a directory. + * @param ecPolicy The erasure coding policy for the directory. + * If null, the default will be used. * @throws IOException */ - public void createErasureCodingZone(final Path path, + public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy) throws IOException { - dfs.createErasureCodingZone(path, ecPolicy); + dfs.setErasureCodingPolicy(path, ecPolicy); } /** - * Get the ErasureCoding zone information for the specified path + * Get the erasure coding policy information for the specified path * * @param path - * @return Returns the zone information if path is in EC zone, null otherwise + * @return Returns the policy information if file or directory on the path is + * erasure coded. Null otherwise. * @throws IOException */ - public ErasureCodingZone getErasureCodingZone(final Path path) + public ErasureCodingPolicy getErasureCodingPolicy(final Path path) throws IOException { - return dfs.getErasureCodingZone(path); + return dfs.getErasureCodingPolicy(path); } /** - * Get the ErasureCoding policies supported. + * Get the Erasure coding policies supported. * * @throws IOException */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java deleted file mode 100644 index 533b630893c15..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.hdfs.protocol; - -/** - * Information about the EC Zone at the specified path. - */ -public class ErasureCodingZone { - - private String dir; - private ErasureCodingPolicy ecPolicy; - - public ErasureCodingZone(String dir, ErasureCodingPolicy ecPolicy) { - this.dir = dir; - this.ecPolicy = ecPolicy; - } - - /** - * Get directory of the EC zone. - * - * @return - */ - public String getDir() { - return dir; - } - - /** - * Get the erasure coding policy for the EC Zone - * - * @return - */ - public ErasureCodingPolicy getErasureCodingPolicy() { - return ecPolicy; - } - - @Override - public String toString() { - return "Dir: " + getDir() + ", Policy: " + ecPolicy; - } -} 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 97445a6bf1dc2..32174849e6423 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 @@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -203,10 +202,10 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyResponseProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto; @@ -1402,14 +1401,14 @@ public ListEncryptionZonesResponseProto listEncryptionZones( } @Override - public CreateErasureCodingZoneResponseProto createErasureCodingZone( - RpcController controller, CreateErasureCodingZoneRequestProto req) + public SetErasureCodingPolicyResponseProto setErasureCodingPolicy( + RpcController controller, SetErasureCodingPolicyRequestProto req) throws ServiceException { try { ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(req .getEcPolicy()) : null; - server.createErasureCodingZone(req.getSrc(), ecPolicy); - return CreateErasureCodingZoneResponseProto.newBuilder().build(); + server.setErasureCodingPolicy(req.getSrc(), ecPolicy); + return SetErasureCodingPolicyResponseProto.newBuilder().build(); } catch (IOException e) { throw new ServiceException(e); } @@ -1554,13 +1553,13 @@ public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcControl } @Override - public GetErasureCodingZoneResponseProto getErasureCodingZone(RpcController controller, - GetErasureCodingZoneRequestProto request) throws ServiceException { + public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller, + GetErasureCodingPolicyRequestProto request) throws ServiceException { try { - ErasureCodingZone ecZone = server.getErasureCodingZone(request.getSrc()); - GetErasureCodingZoneResponseProto.Builder builder = GetErasureCodingZoneResponseProto.newBuilder(); - if (ecZone != null) { - builder.setECZone(PBHelper.convertErasureCodingZone(ecZone)); + ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc()); + GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder(); + if (ecPolicy != null) { + builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy)); } return builder.build(); } catch (IOException 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 c8c468ecfca8a..841924464ea6d 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 @@ -58,7 +58,6 @@ 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.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -168,9 +167,9 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; @@ -1420,17 +1419,17 @@ public BatchedEntries listEncryptionZones(long id) } @Override - public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) + public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) throws IOException { - final CreateErasureCodingZoneRequestProto.Builder builder = - CreateErasureCodingZoneRequestProto.newBuilder(); + final SetErasureCodingPolicyRequestProto.Builder builder = + SetErasureCodingPolicyRequestProto.newBuilder(); builder.setSrc(src); if (ecPolicy != null) { builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy)); } - CreateErasureCodingZoneRequestProto req = builder.build(); + SetErasureCodingPolicyRequestProto req = builder.build(); try { - rpcProxy.createErasureCodingZone(null, req); + rpcProxy.setErasureCodingPolicy(null, req); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } @@ -1577,14 +1576,14 @@ public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { } @Override - public ErasureCodingZone getErasureCodingZone(String src) throws IOException { - GetErasureCodingZoneRequestProto req = GetErasureCodingZoneRequestProto.newBuilder() + public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException { + GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder() .setSrc(src).build(); try { - GetErasureCodingZoneResponseProto response = rpcProxy.getErasureCodingZone( + GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy( null, req); - if (response.hasECZone()) { - return PBHelper.convertErasureCodingZone(response.getECZone()); + if (response.hasEcPolicy()) { + return PBHelper.convertErasureCodingPolicy(response.getEcPolicy()); } return null; } catch (ServiceException 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 f419c46ef3eb4..ce39e152f96e2 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 @@ -75,7 +75,6 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.fs.FileEncryptionInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -132,7 +131,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto; import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto; -import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; @@ -2975,19 +2973,6 @@ public static ErasureCodingPolicyProto convertErasureCodingPolicy( .setCellSize(policy.getCellSize()); return builder.build(); } - - public static ErasureCodingZoneProto convertErasureCodingZone( - ErasureCodingZone ecZone) { - return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir()) - .setEcPolicy(convertErasureCodingPolicy(ecZone.getErasureCodingPolicy())) - .build(); - } - - public static ErasureCodingZone convertErasureCodingZone( - ErasureCodingZoneProto ecZoneProto) { - return new ErasureCodingZone(ecZoneProto.getDir(), - convertErasureCodingPolicy(ecZoneProto.getEcPolicy())); - } public static BlockECRecoveryInfo convertBlockECRecoveryInfo( BlockECRecoveryInfoProto blockEcRecoveryInfoProto) { 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 8232ab9c9ff37..3c1c4612e3e0b 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 @@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; @@ -924,11 +923,9 @@ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, final boolean isFileUnderConstruction, final long offset, final long length, final boolean needBlockToken, final boolean inSnapshot, FileEncryptionInfo feInfo, - ErasureCodingZone ecZone) + ErasureCodingPolicy ecPolicy) throws IOException { assert namesystem.hasReadLock(); - final ErasureCodingPolicy ecPolicy = ecZone != null ? ecZone - .getErasureCodingPolicy() : null; if (blocks == null) { return null; } else if (blocks.length == 0) { @@ -1607,14 +1604,14 @@ private boolean validateRecoveryWork(BlockRecoveryWork rw) { assert rw instanceof ErasureCodingWork; assert rw.getTargets().length > 0; String src = getBlockCollection(block).getName(); - ErasureCodingZone ecZone = null; + ErasureCodingPolicy ecPolicy = null; try { - ecZone = namesystem.getErasureCodingZoneForPath(src); + ecPolicy = namesystem.getErasureCodingPolicyForPath(src); } catch (IOException e) { blockLog - .warn("Failed to get the EC zone for the file {} ", src); + .warn("Failed to get EC policy for the file {} ", src); } - if (ecZone == null) { + if (ecPolicy == null) { blockLog.warn("No erasure coding policy found for the file {}. " + "So cannot proceed for recovery", src); // TODO: we may have to revisit later for what we can do better to @@ -1624,8 +1621,7 @@ private boolean validateRecoveryWork(BlockRecoveryWork rw) { rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded( new ExtendedBlock(namesystem.getBlockPoolId(), block), rw.getSrcNodes(), rw.getTargets(), - ((ErasureCodingWork) rw).getLiveBlockIndicies(), - ecZone.getErasureCodingPolicy()); + ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy); } else { rw.getSrcNodes()[0].addBlockToBeReplicated(block, targets); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java index da5271710d4b4..eb8ea8a0b21ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java @@ -387,8 +387,8 @@ enum BlockUCState { "raw.hdfs.crypto.file.encryption.info"; String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER = "security.hdfs.unreadable.by.superuser"; - String XATTR_ERASURECODING_ZONE = - "raw.hdfs.erasurecoding.zone"; + String XATTR_ERASURECODING_POLICY = + "raw.hdfs.erasurecoding.policy"; long BLOCK_GROUP_INDEX_MASK = 15; byte MAX_BLOCKS_IN_GROUP = 16; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java index 71ac36a79bbd9..b77279b4f0f97 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java @@ -83,7 +83,7 @@ public static ErasureCodingPolicy[] getSystemPolices() { /** * Get system-wide default policy, which can be used by default - * when no policy is specified for an EC zone. + * when no policy is specified for a path. * @return ecPolicy */ public static ErasureCodingPolicy getSystemDefaultPolicy() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java deleted file mode 100644 index a0ac033b9be4d..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java +++ /dev/null @@ -1,163 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hdfs.server.namenode; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; - -import org.apache.hadoop.fs.XAttr; -import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.hdfs.XAttrHelper; -import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.WritableUtils; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.EnumSet; -import java.util.List; - -import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE; - -/** - * Manages the list of erasure coding zones in the filesystem. - *

    - * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory - * lock being held for many operations. The FSDirectory lock should not be - * taken if the manager lock is already held. - * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager} - */ -public class ErasureCodingZoneManager { - private final FSDirectory dir; - - /** - * Construct a new ErasureCodingZoneManager. - * - * @param dir Enclosing FSDirectory - */ - public ErasureCodingZoneManager(FSDirectory dir) { - this.dir = dir; - } - - ErasureCodingPolicy getErasureCodingPolicy(INodesInPath iip) throws IOException { - ErasureCodingZone ecZone = getErasureCodingZone(iip); - return ecZone == null ? null : ecZone.getErasureCodingPolicy(); - } - - ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException { - assert dir.hasReadLock(); - Preconditions.checkNotNull(iip, "INodes cannot be null"); - List inodes = iip.getReadOnlyINodes(); - for (int i = inodes.size() - 1; i >= 0; i--) { - final INode inode = inodes.get(i); - if (inode == null) { - continue; - } - // We don't allow symlinks in an EC zone, or pointing to a file/dir in - // an EC. Therefore if a symlink is encountered, the dir shouldn't have - // EC - // TODO: properly support symlinks in EC zones - if (inode.isSymlink()) { - return null; - } - final List xAttrs = inode.getXAttrFeature() == null ? - new ArrayList(0) - : inode.getXAttrFeature().getXAttrs(); - for (XAttr xAttr : xAttrs) { - if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixedName(xAttr))) { - ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue()); - DataInputStream dIn=new DataInputStream(bIn); - String ecPolicyName = WritableUtils.readString(dIn); - ErasureCodingPolicy ecPolicy = dir.getFSNamesystem() - .getErasureCodingPolicyManager().getPolicy(ecPolicyName); - return new ErasureCodingZone(dir.getInode(inode.getId()) - .getFullPathName(), ecPolicy); - } - } - } - return null; - } - - List createErasureCodingZone(final INodesInPath srcIIP, - ErasureCodingPolicy ecPolicy) throws IOException { - assert dir.hasWriteLock(); - Preconditions.checkNotNull(srcIIP, "INodes cannot be null"); - String src = srcIIP.getPath(); - if (dir.isNonEmptyDirectory(srcIIP)) { - throw new IOException( - "Attempt to create an erasure coding zone for a " + - "non-empty directory " + src); - } - if (srcIIP.getLastINode() != null && - !srcIIP.getLastINode().isDirectory()) { - throw new IOException("Attempt to create an erasure coding zone " + - "for a file " + src); - } - if (getErasureCodingPolicy(srcIIP) != null) { - throw new IOException("Directory " + src + " is already in an " + - "erasure coding zone."); - } - - // System default erasure coding policy will be used since no specified. - if (ecPolicy == null) { - ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); - } - - final XAttr ecXAttr; - DataOutputStream dOut = null; - try { - ByteArrayOutputStream bOut = new ByteArrayOutputStream(); - dOut = new DataOutputStream(bOut); - WritableUtils.writeString(dOut, ecPolicy.getName()); - ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE, - bOut.toByteArray()); - } finally { - IOUtils.closeStream(dOut); - } - final List xattrs = Lists.newArrayListWithCapacity(1); - xattrs.add(ecXAttr); - FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs, - EnumSet.of(XAttrSetFlag.CREATE)); - return xattrs; - } - - void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src) - throws IOException { - assert dir.hasReadLock(); - final ErasureCodingZone srcZone = getErasureCodingZone(srcIIP); - final ErasureCodingZone dstZone = getErasureCodingZone(dstIIP); - if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) { - return; - } - final ErasureCodingPolicy srcECPolicy = - srcZone != null ? srcZone.getErasureCodingPolicy() : null; - final ErasureCodingPolicy dstECPolicy = - dstZone != null ? dstZone.getErasureCodingPolicy() : null; - if (srcECPolicy != null && !srcECPolicy.equals(dstECPolicy) || - dstECPolicy != null && !dstECPolicy.equals(srcECPolicy)) { - throw new IOException( - src + " can't be moved because the source and destination have " + - "different erasure coding policies."); - } - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java index 8c515d1641613..4162760d2a59b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java @@ -17,14 +17,27 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; +import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.WritableUtils; + +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_POLICY; /** * Helper class to perform erasure coding related operations. @@ -38,18 +51,17 @@ final class FSDirErasureCodingOp { private FSDirErasureCodingOp() {} /** - * Create an erasure coding zone on directory src. + * Set an erasure coding policy on the given path. * - * @param fsn namespace - * @param srcArg the path of a directory which will be the root of the - * erasure coding zone. The directory must be empty. - * @param ecPolicy erasure coding policy for the erasure coding zone + * @param fsn The namespace + * @param srcArg The path of the target directory. + * @param ecPolicy The erasure coding policy to set on the target directory. * @param logRetryCache whether to record RPC ids in editlog for retry * cache rebuilding * @return {@link HdfsFileStatus} * @throws IOException */ - static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn, + static HdfsFileStatus setErasureCodingPolicy(final FSNamesystem fsn, final String srcArg, final ErasureCodingPolicy ecPolicy, final boolean logRetryCache) throws IOException { assert fsn.hasWriteLock(); @@ -66,8 +78,7 @@ static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn, fsd.writeLock(); try { iip = fsd.getINodesInPath4Write(src, false); - xAttrs = fsn.getErasureCodingZoneManager().createErasureCodingZone( - iip, ecPolicy); + xAttrs = createErasureCodingPolicyXAttr(fsn, iip, ecPolicy); } finally { fsd.writeUnlock(); } @@ -75,62 +86,83 @@ static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn, return fsd.getAuditFileInfo(iip); } - /** - * Get the erasure coding zone information for specified path. - * - * @param fsn namespace - * @param src path - * @return {@link ErasureCodingZone} - * @throws IOException - */ - static ErasureCodingZone getErasureCodingZone(final FSNamesystem fsn, - final String src) throws IOException { - assert fsn.hasReadLock(); + static List createErasureCodingPolicyXAttr(final FSNamesystem fsn, + final INodesInPath srcIIP, ErasureCodingPolicy ecPolicy) throws IOException { + FSDirectory fsd = fsn.getFSDirectory(); + assert fsd.hasWriteLock(); + Preconditions.checkNotNull(srcIIP, "INodes cannot be null"); + String src = srcIIP.getPath(); + if (srcIIP.getLastINode() != null && + !srcIIP.getLastINode().isDirectory()) { + throw new IOException("Attempt to set an erasure coding policy " + + "for a file " + src); + } + if (getErasureCodingPolicyForPath(fsn, srcIIP) != null) { + throw new IOException("Directory " + src + " already has an " + + "erasure coding policy."); + } - final INodesInPath iip = getINodesInPath(fsn, src); - return getErasureCodingZoneForPath(fsn, iip); + // System default erasure coding policy will be used since no specified. + if (ecPolicy == null) { + ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + } + + final XAttr ecXAttr; + DataOutputStream dOut = null; + try { + ByteArrayOutputStream bOut = new ByteArrayOutputStream(); + dOut = new DataOutputStream(bOut); + WritableUtils.writeString(dOut, ecPolicy.getName()); + ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_POLICY, + bOut.toByteArray()); + } finally { + IOUtils.closeStream(dOut); + } + final List xattrs = Lists.newArrayListWithCapacity(1); + xattrs.add(ecXAttr); + FSDirXAttrOp.unprotectedSetXAttrs(fsd, src, xattrs, + EnumSet.of(XAttrSetFlag.CREATE)); + return xattrs; } /** - * Get erasure coding zone information for specified path. + * Get the erasure coding policy information for specified path. * * @param fsn namespace - * @param iip inodes in the path containing the file - * @return {@link ErasureCodingZone} + * @param src path + * @return {@link ErasureCodingPolicy} * @throws IOException */ - static ErasureCodingZone getErasureCodingZone(final FSNamesystem fsn, - final INodesInPath iip) throws IOException { + static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn, + final String src) throws IOException { assert fsn.hasReadLock(); - return getErasureCodingZoneForPath(fsn, iip); + final INodesInPath iip = getINodesInPath(fsn, src); + return getErasureCodingPolicyForPath(fsn, iip); } /** - * Check if the file is in erasure coding zone. + * Check if the file or directory has an erasure coding policy. * * @param fsn namespace * @param srcArg path - * @return true represents the file is in erasure coding zone, false otw + * @return Whether the file or directory has an erasure coding policy. * @throws IOException */ - static boolean isInErasureCodingZone(final FSNamesystem fsn, + static boolean hasErasureCodingPolicy(final FSNamesystem fsn, final String srcArg) throws IOException { - assert fsn.hasReadLock(); - - final INodesInPath iip = getINodesInPath(fsn, srcArg); - return getErasureCodingPolicyForPath(fsn, iip) != null; + return hasErasureCodingPolicy(fsn, getINodesInPath(fsn, srcArg)); } /** - * Check if the file is in erasure coding zone. + * Check if the file or directory has an erasure coding policy. * * @param fsn namespace * @param iip inodes in the path containing the file - * @return true represents the file is in erasure coding zone, false otw + * @return Whether the file or directory has an erasure coding policy. * @throws IOException */ - static boolean isInErasureCodingZone(final FSNamesystem fsn, + static boolean hasErasureCodingPolicy(final FSNamesystem fsn, final INodesInPath iip) throws IOException { return getErasureCodingPolicy(fsn, iip) != null; } @@ -178,25 +210,46 @@ private static INodesInPath getINodesInPath(final FSNamesystem fsn, return iip; } - private static ErasureCodingZone getErasureCodingZoneForPath( - final FSNamesystem fsn, final INodesInPath iip) throws IOException { - final FSDirectory fsd = fsn.getFSDirectory(); - fsd.readLock(); - try { - return fsn.getErasureCodingZoneManager().getErasureCodingZone(iip); - } finally { - fsd.readUnlock(); - } - } - - private static ErasureCodingPolicy getErasureCodingPolicyForPath(final FSNamesystem fsn, - final INodesInPath iip) throws IOException { - final FSDirectory fsd = fsn.getFSDirectory(); + private static ErasureCodingPolicy getErasureCodingPolicyForPath(FSNamesystem fsn, + INodesInPath iip) throws IOException { + Preconditions.checkNotNull(iip, "INodes cannot be null"); + FSDirectory fsd = fsn.getFSDirectory(); fsd.readLock(); try { - return fsn.getErasureCodingZoneManager().getErasureCodingPolicy(iip); + List inodes = iip.getReadOnlyINodes(); + for (int i = inodes.size() - 1; i >= 0; i--) { + final INode inode = inodes.get(i); + if (inode == null) { + continue; + } + /** + * TODO: lookup {@link ErasureCodingPolicyManager#getSystemPolices()} + */ + if (inode.isFile()) { + return inode.asFile().getErasureCodingPolicyID() == 0 ? + null : ErasureCodingPolicyManager.getSystemDefaultPolicy(); + } + // We don't allow setting EC policies on paths with a symlink. Thus + // if a symlink is encountered, the dir shouldn't have EC policy. + // TODO: properly support symlinks + if (inode.isSymlink()) { + return null; + } + final XAttrFeature xaf = inode.getXAttrFeature(); + if (xaf != null) { + XAttr xattr = xaf.getXAttr(XATTR_ERASURECODING_POLICY); + if (xattr != null) { + ByteArrayInputStream bIn = new ByteArrayInputStream(xattr.getValue()); + DataInputStream dIn = new DataInputStream(bIn); + String ecPolicyName = WritableUtils.readString(dIn); + return fsd.getFSNamesystem().getErasureCodingPolicyManager(). + getPolicy(ecPolicyName); + } + } + } } finally { fsd.readUnlock(); } + return null; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java index ccdef1f3e26d6..b01570848a2c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java @@ -185,7 +185,6 @@ static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst, } fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src); - fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src); // Ensure dst has quota to accommodate rename verifyFsLimitsForRename(fsd, srcIIP, dstIIP); verifyQuotaForRename(fsd, srcIIP, dstIIP); @@ -358,7 +357,6 @@ static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst, BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite(); fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src); - fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src); final INode dstInode = dstIIP.getLastINode(); List snapshottableDirs = new ArrayList<>(); if (dstInode != null) { // Destination exists diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index 6ec97c90a9620..e178c685d257f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -182,12 +181,12 @@ static GetBlockLocationsResult getBlockLocations( final FileEncryptionInfo feInfo = isReservedName ? null : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip); - final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( - fsd.getFSNamesystem(), iip); + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp. + getErasureCodingPolicy(fsd.getFSNamesystem(), iip); final LocatedBlocks blocks = bm.createLocatedBlocks( inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset, - length, needBlockToken, iip.isSnapshot(), feInfo, ecZone); + length, needBlockToken, iip.isSnapshot(), feInfo, ecPolicy); // Set caching information for the located blocks. for (LocatedBlock lb : blocks.getLocatedBlocks()) { @@ -447,10 +446,8 @@ static HdfsFileStatus createFileStatus( final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); - final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsd.getFSNamesystem(), iip); - final ErasureCodingPolicy ecPolicy = - ecZone != null ? ecZone.getErasureCodingPolicy() : null; if (node.isFile()) { final INodeFile fileNode = node.asFile(); @@ -505,7 +502,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( final boolean isEncrypted; final FileEncryptionInfo feInfo = isRawPath ? null : fsd.getFileEncryptionInfo(node, snapshot, iip); - final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsd.getFSNamesystem(), iip); if (node.isFile()) { final INodeFile fileNode = node.asFile(); @@ -520,7 +517,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( loc = fsd.getBlockManager().createLocatedBlocks( fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false, - inSnapshot, feInfo, ecZone); + inSnapshot, feInfo, ecPolicy); if (loc == null) { loc = new LocatedBlocks(); } @@ -531,8 +528,6 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( } int childrenNum = node.isDirectory() ? node.asDirectory().getChildrenNum(snapshot) : 0; - final ErasureCodingPolicy ecPolicy = - ecZone != null ? ecZone.getErasureCodingPolicy() : null; HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(size, node.isDirectory(), replication, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index e480959d03785..867b4515469c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -486,8 +485,8 @@ static INodeFile addFileForEditLog( Preconditions.checkNotNull(existing); assert fsd.hasWriteLock(); try { - // check if the file is in an EC zone - final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone( + // check if the file has an EC policy + final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy( fsd.getFSNamesystem(), existing); if (underConstruction) { newNode = newINodeFile(id, permissions, modificationTime, @@ -533,9 +532,8 @@ private static BlockInfo addBlock(FSDirectory fsd, String path, // associate new last block for the file final BlockInfo blockInfo; if (isStriped) { - ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsd.getFSNamesystem(), inodesInPath); - ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy(); short numDataUnits = (short) ecPolicy.getNumDataUnits(); short numParityUnits = (short) ecPolicy.getNumParityUnits(); short numLocations = (short) (numDataUnits + numParityUnits); @@ -586,7 +584,7 @@ private static INodesInPath addFile( INodesInPath newiip; fsd.writeLock(); try { - final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone( + final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy( fsd.getFSNamesystem(), existing); INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions, modTime, modTime, replication, preferredBlockSize, isStriped); 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 c9fb6cd922637..6538aba77f2a9 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 @@ -215,9 +215,6 @@ public int getWriteHoldCount() { @VisibleForTesting public final EncryptionZoneManager ezManager; - @VisibleForTesting - public final ErasureCodingZoneManager ecZoneManager; - /** * Caches frequently used file names used in {@link INode} to reuse * byte[] objects and reduce heap usage. @@ -314,7 +311,6 @@ public int getWriteHoldCount() { namesystem = ns; this.editLog = ns.getEditLog(); ezManager = new EncryptionZoneManager(this, conf); - ecZoneManager = new ErasureCodingZoneManager(this); } FSNamesystem getFSNamesystem() { 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 68324f3f35882..169154885c4b9 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 @@ -36,7 +36,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; @@ -416,9 +416,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsDir.getFSNamesystem(), iip); - updateBlocks(fsDir, addCloseOp, iip, newFile, ecZone); + updateBlocks(fsDir, addCloseOp, iip, newFile, ecPolicy); break; } case OP_CLOSE: { @@ -438,9 +438,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // Update the salient file attributes. file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID); file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID); - ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsDir.getFSNamesystem(), iip); - updateBlocks(fsDir, addCloseOp, iip, file, ecZone); + updateBlocks(fsDir, addCloseOp, iip, file, ecPolicy); // Now close the file if (!file.isUnderConstruction() && @@ -498,9 +498,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures - ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsDir.getFSNamesystem(), iip); - updateBlocks(fsDir, updateOp, iip, oldFile, ecZone); + updateBlocks(fsDir, updateOp, iip, oldFile, ecPolicy); if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); @@ -517,9 +517,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // add the new block to the INodeFile - ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone( + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( fsDir.getFSNamesystem(), iip); - addNewBlock(addBlockOp, oldFile, ecZone); + addNewBlock(addBlockOp, oldFile, ecPolicy); break; } case OP_SET_REPLICATION: { @@ -961,7 +961,7 @@ private static String formatEditLogReplayError(EditLogInputStream in, * Add a new block into the given INodeFile */ private void addNewBlock(AddBlockOp op, INodeFile file, - ErasureCodingZone ecZone) throws IOException { + ErasureCodingPolicy ecPolicy) throws IOException { BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); Block newBlock= op.getLastBlock(); @@ -988,10 +988,9 @@ private void addNewBlock(AddBlockOp op, INodeFile file, } // add the new block final BlockInfo newBlockInfo; - boolean isStriped = ecZone != null; + boolean isStriped = ecPolicy != null; if (isStriped) { - newBlockInfo = new BlockInfoStriped(newBlock, - ecZone.getErasureCodingPolicy()); + newBlockInfo = new BlockInfoStriped(newBlock, ecPolicy); } else { newBlockInfo = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); @@ -1008,7 +1007,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file, * @throws IOException */ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, - INodesInPath iip, INodeFile file, ErasureCodingZone ecZone) + INodesInPath iip, INodeFile file, ErasureCodingPolicy ecPolicy) throws IOException { // Update its block list BlockInfo[] oldBlocks = file.getBlocks(); @@ -1068,7 +1067,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, throw new IOException("Trying to delete non-existant block " + oldBlock); } } else if (newBlocks.length > oldBlocks.length) { - final boolean isStriped = ecZone != null; + final boolean isStriped = ecPolicy != null; // We're adding blocks for (int i = oldBlocks.length; i < newBlocks.length; i++) { Block newBlock = newBlocks[i]; @@ -1078,8 +1077,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, // what about an old-version fsync() where fsync isn't called // until several blocks in? if (isStriped) { - newBI = new BlockInfoStriped(newBlock, - ecZone.getErasureCodingPolicy()); + newBI = new BlockInfoStriped(newBlock, ecPolicy); } else { newBI = new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication()); 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 5f394460ecb19..ed52ca447b753 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 @@ -177,7 +177,6 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; @@ -187,7 +186,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; -import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.protocol.RollingUpgradeException; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; @@ -2133,7 +2131,7 @@ private HdfsFileStatus startFileInt(final String src, readLock(); try { checkOperation(OperationCategory.READ); - if (!FSDirErasureCodingOp.isInErasureCodingZone(this, src)) { + if (!FSDirErasureCodingOp.hasErasureCodingPolicy(this, src)) { blockManager.verifyReplication(src, replication, clientMachine); } } finally { @@ -3206,9 +3204,8 @@ void commitOrCompleteLastBlock( final long diff; final short replicationFactor; if (fileINode.isStriped()) { - final ErasureCodingZone ecZone = FSDirErasureCodingOp - .getErasureCodingZone(this, iip); - final ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy(); + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp + .getErasureCodingPolicy(this, iip); final short numDataUnits = (short) ecPolicy.getNumDataUnits(); final short numParityUnits = (short) ecPolicy.getNumParityUnits(); @@ -6241,11 +6238,6 @@ public ErasureCodingPolicyManager getErasureCodingPolicyManager() { return ecPolicyManager; } - /** @return the ErasureCodingZoneManager. */ - public ErasureCodingZoneManager getErasureCodingZoneManager() { - return dir.ecZoneManager; - } - @Override // NameNodeMXBean public String getCorruptFiles() { List list = new ArrayList(); @@ -7192,15 +7184,14 @@ BatchedListEntries listEncryptionZones(long prevId) } /** - * Create an erasure coding zone on directory src. - * @param srcArg the path of a directory which will be the root of the - * erasure coding zone. The directory must be empty. - * @param ecPolicy erasure coding policy for the erasure coding zone + * Set an erasure coding policy on the given path. + * @param srcArg The path of the target directory. + * @param ecPolicy The erasure coding policy to set on the target directory. * @throws AccessControlException if the caller is not the superuser. * @throws UnresolvedLinkException if the path can't be resolved. * @throws SafeModeException if the Namenode is in safe mode. */ - void createErasureCodingZone(final String srcArg, final ErasureCodingPolicy + void setErasureCodingPolicy(final String srcArg, final ErasureCodingPolicy ecPolicy, final boolean logRetryCache) throws IOException, UnresolvedLinkException, SafeModeException, AccessControlException { checkSuperuserPrivilege(); @@ -7210,8 +7201,8 @@ void createErasureCodingZone(final String srcArg, final ErasureCodingPolicy writeLock(); try { checkOperation(OperationCategory.WRITE); - checkNameNodeSafeMode("Cannot create erasure coding zone on " + srcArg); - resultingStat = FSDirErasureCodingOp.createErasureCodingZone(this, + checkNameNodeSafeMode("Cannot set erasure coding policy on " + srcArg); + resultingStat = FSDirErasureCodingOp.setErasureCodingPolicy(this, srcArg, ecPolicy, logRetryCache); success = true; } finally { @@ -7219,21 +7210,21 @@ void createErasureCodingZone(final String srcArg, final ErasureCodingPolicy if (success) { getEditLog().logSync(); } - logAuditEvent(success, "createErasureCodingZone", srcArg, null, + logAuditEvent(success, "setErasureCodingPolicy", srcArg, null, resultingStat); } } /** - * Get the erasure coding zone information for specified path + * Get the erasure coding policy information for specified path */ - ErasureCodingZone getErasureCodingZone(String src) + ErasureCodingPolicy getErasureCodingPolicy(String src) throws AccessControlException, UnresolvedLinkException, IOException { checkOperation(OperationCategory.READ); readLock(); try { checkOperation(OperationCategory.READ); - return getErasureCodingZoneForPath(src); + return getErasureCodingPolicyForPath(src); } finally { readUnlock(); } @@ -7461,9 +7452,9 @@ public String getTotalSyncTimes() { } @Override - public ErasureCodingZone getErasureCodingZoneForPath(String src) + public ErasureCodingPolicy getErasureCodingPolicyForPath(String src) throws IOException { - return FSDirErasureCodingOp.getErasureCodingZone(this, src); + return FSDirErasureCodingOp.getErasureCodingPolicy(this, src); } } 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 ae9b0d2e7f3a6..6f7b702bb5a78 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 @@ -437,6 +437,20 @@ public final void setStoragePolicyID(byte storagePolicyId, setStoragePolicyID(storagePolicyId); } + + /** + * @return The ID of the erasure coding policy on the file. 0 represents no + * EC policy (file is in contiguous format). 1 represents the system + * default EC policy: + * {@link ErasureCodingPolicyManager#SYS_DEFAULT_POLICY}. + * TODO: support more policies by reusing {@link HeaderFormat#REPLICATION}. + */ + @VisibleForTesting + @Override + public byte getErasureCodingPolicyID() { + return isStriped() ? (byte)1 : (byte)0; + } + /** * @return true if the file is in the striping layout. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java index 13bd9e962e3fe..8cd481a68e063 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java @@ -20,7 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat; -import org.apache.hadoop.hdfs.server.namenode.XAttrFeature; + /** * The attributes of a file. */ @@ -32,6 +32,9 @@ public interface INodeFileAttributes extends INodeAttributes { /** @return whether the file is striped (instead of contiguous) */ public boolean isStriped(); + /** @return whether the file is striped (instead of contiguous) */ + public byte getErasureCodingPolicyID(); + /** @return preferred block size in bytes */ public long getPreferredBlockSize(); @@ -77,6 +80,11 @@ public boolean isStriped() { return HeaderFormat.isStriped(header); } + @Override + public byte getErasureCodingPolicyID() { + return isStriped() ? (byte)1 : (byte)0; + } + @Override public long getPreferredBlockSize() { return HeaderFormat.getPreferredBlockSize(header); 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 ab29e4de634b0..41439641a81bd 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 @@ -85,7 +85,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FSLimitException; @@ -1423,8 +1422,8 @@ public void refreshServiceAcl() throws IOException { @Override // RefreshAuthorizationPolicyProtocol public void refreshUserToGroupsMappings() throws IOException { - LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + - getRemoteUser().getShortUserName()); + LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + + getRemoteUser().getShortUserName()); Groups.getUserToGroupsMappingService().refresh(); } @@ -1557,7 +1556,7 @@ public String createSnapshot(String snapshotRoot, String snapshotName) } namesystem.checkOperation(OperationCategory.WRITE); CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache, - null); + null); if (cacheEntry != null && cacheEntry.isSuccess()) { return (String) cacheEntry.getPayload(); } @@ -1849,7 +1848,7 @@ public BatchedEntries listEncryptionZones( } @Override // ClientProtocol - public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) + public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) throws IOException { checkNNStartup(); final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); @@ -1858,7 +1857,7 @@ public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy) } boolean success = false; try { - namesystem.createErasureCodingZone(src, ecPolicy, cacheEntry != null); + namesystem.setErasureCodingPolicy(src, ecPolicy, cacheEntry != null); success = true; } finally { RetryCache.setState(cacheEntry, success); @@ -2068,8 +2067,8 @@ public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { } @Override // ClientProtocol - public ErasureCodingZone getErasureCodingZone(String src) throws IOException { + public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException { checkNNStartup(); - return namesystem.getErasureCodingZone(src); + return namesystem.getErasureCodingPolicy(src); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java index 923a33536c4d8..e1702d9e5c872 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java @@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; @@ -52,16 +52,16 @@ public interface Namesystem extends RwLock, SafeMode { void checkOperation(OperationCategory read) throws StandbyException; /** - * Gets the ECZone for path + * Gets the erasure coding policy for the path * @param src * - path - * @return {@link ErasureCodingZone} + * @return {@link ErasureCodingPolicy} * @throws IOException */ - ErasureCodingZone getErasureCodingZoneForPath(String src) + ErasureCodingPolicy getErasureCodingPolicyForPath(String src) throws IOException; boolean isInSnapshot(BlockInfo blockUC); CacheManager getCacheManager(); -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index f3260dab48bf0..b857d3e66353e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -30,7 +30,6 @@ import org.apache.hadoop.fs.shell.CommandFactory; import org.apache.hadoop.fs.shell.PathData; import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.protocol.ErasureCodingZone; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.util.StringUtils; @@ -45,9 +44,9 @@ public abstract class ECCommand extends Command { public static void registerCommands(CommandFactory factory) { // Register all commands of Erasure CLI, with a '-' at the beginning in name // of the command. - factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME); - factory.addClass(GetECZoneCommand.class, "-" - + GetECZoneCommand.NAME); + factory.addClass(SetECPolicyCommand.class, "-" + SetECPolicyCommand.NAME); + factory.addClass(GetECPolicyCommand.class, "-" + + GetECPolicyCommand.NAME); factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME); } @@ -76,17 +75,18 @@ protected void processPath(PathData item) throws IOException { } /** - * A command to create an EC zone for a path, with a erasure coding policy name. + * A command to set the erasure coding policy for a directory, with the name + * of the policy. */ - static class CreateECZoneCommand extends ECCommand { - public static final String NAME = "createZone"; + static class SetECPolicyCommand extends ECCommand { + public static final String NAME = "setPolicy"; public static final String USAGE = "[-s ] "; public static final String DESCRIPTION = - "Create a zone to encode files using a specified policy\n" + "Set a specified erasure coding policy to a directory\n" + "Options :\n" + " -s : erasure coding policy name to encode files. " + "If not passed the default policy will be used\n" - + " : Path to an empty directory. Under this directory " + + " : Path to a directory. Under this directory " + "files will be encoded using specified erasure coding policy"; private String ecPolicyName; private ErasureCodingPolicy ecPolicy = null; @@ -129,23 +129,23 @@ protected void processPath(PathData item) throws IOException { throw new HadoopIllegalArgumentException(sb.toString()); } } - dfs.createErasureCodingZone(item.path, ecPolicy); - out.println("EC Zone created successfully at " + item.path); + dfs.setErasureCodingPolicy(item.path, ecPolicy); + out.println("EC policy set successfully at " + item.path); } catch (IOException e) { - throw new IOException("Unable to create EC zone for the path " + throw new IOException("Unable to set EC policy for the path " + item.path + ". " + e.getMessage()); } } } /** - * Get the information about the zone + * Get the erasure coding policy of a file or directory */ - static class GetECZoneCommand extends ECCommand { - public static final String NAME = "getZone"; + static class GetECPolicyCommand extends ECCommand { + public static final String NAME = "getPolicy"; public static final String USAGE = ""; public static final String DESCRIPTION = - "Get information about the EC zone at specified path\n"; + "Get erasure coding policy information about at specified path\n"; @Override protected void processOptions(LinkedList args) throws IOException { @@ -162,14 +162,14 @@ protected void processPath(PathData item) throws IOException { super.processPath(item); DistributedFileSystem dfs = (DistributedFileSystem) item.fs; try { - ErasureCodingZone ecZone = dfs.getErasureCodingZone(item.path); - if (ecZone != null) { - out.println(ecZone.toString()); + ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(item.path); + if (ecPolicy != null) { + out.println(ecPolicy.toString()); } else { - out.println("Path " + item.path + " is not in EC zone"); + out.println("Path " + item.path + " is not erasure coded."); } } catch (IOException e) { - throw new IOException("Unable to get EC zone for the path " + throw new IOException("Unable to get EC policy for the path " + item.path + ". " + e.getMessage()); } } 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 6500b963871e0..3551055cdf618 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 @@ -1892,12 +1892,12 @@ public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock( public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir, int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception { DistributedFileSystem dfs = cluster.getFileSystem(); - // If outer test already created EC zone, dir should be left as null + // If outer test already set EC policy, dir should be left as null if (toMkdir) { assert dir != null; dfs.mkdirs(dir); try { - dfs.getClient().createErasureCodingZone(dir.toString(), null); + dfs.getClient().setErasureCodingPolicy(dir.toString(), null); } catch (IOException e) { if (!e.getMessage().contains("non-empty directory")) { throw e; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index a09f0f0ccfa9e..6fcf644dd8be8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -80,7 +80,7 @@ public void setup() throws IOException { } fs = cluster.getFileSystem(); fs.mkdirs(dirPath); - fs.getClient().createErasureCodingZone(dirPath.toString(), null); + fs.getClient().setErasureCodingPolicy(dirPath.toString(), null); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index a467f40d8b6d1..4263ffaf28993 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -68,7 +68,7 @@ public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index 2aa8484419bef..afb7f95cbb0bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -118,7 +118,7 @@ private void setup(Configuration conf) throws IOException { cluster.waitActive(); dfs = cluster.getFileSystem(); dfs.mkdirs(dir); - dfs.createErasureCodingZone(dir, null); + dfs.setErasureCodingPolicy(dir, null); } private void tearDown() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java similarity index 67% rename from hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java rename to hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java index b68aab99563b7..f60d77d8bd5dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java @@ -35,7 +35,7 @@ import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.junit.Assert.*; -public class TestErasureCodingZones { +public class TestErasureCodingPolicies { private Configuration conf; private MiniDFSCluster cluster; private DistributedFileSystem fs; @@ -59,52 +59,57 @@ public void shutdownCluster() throws IOException { } @Test - public void testCreateECZone() + public void testBasicSetECPolicy() throws IOException, InterruptedException { final Path testDir = new Path("/ec"); fs.mkdir(testDir, FsPermission.getDirDefault()); - /* Normal creation of an erasure coding zone */ - fs.getClient().createErasureCodingZone(testDir.toString(), null); + /* Normal creation of an erasure coding directory */ + fs.getClient().setErasureCodingPolicy(testDir.toString(), null); - /* Verify files under the zone are striped */ + /* Verify files under the directory are striped */ final Path ECFilePath = new Path(testDir, "foo"); fs.create(ECFilePath); INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString()); assertTrue(inode.asFile().isStriped()); - /* Verify that EC zone cannot be created on non-empty dir */ + /** + * Verify that setting EC policy on non-empty directory only affects + * newly created files under the directory. + */ final Path notEmpty = new Path("/nonEmpty"); fs.mkdir(notEmpty, FsPermission.getDirDefault()); - fs.create(new Path(notEmpty, "foo")); + final Path oldFile = new Path(notEmpty, "old"); + fs.create(oldFile); + fs.getClient().setErasureCodingPolicy(notEmpty.toString(), null); + final Path newFile = new Path(notEmpty, "new"); + fs.create(newFile); + INode oldInode = namesystem.getFSDirectory().getINode(oldFile.toString()); + assertFalse(oldInode.asFile().isStriped()); + INode newInode = namesystem.getFSDirectory().getINode(newFile.toString()); + assertTrue(newInode.asFile().isStriped()); + + /* Verify that nested EC policies not supported */ + final Path dir1 = new Path("/dir1"); + final Path dir2 = new Path(dir1, "dir2"); + fs.mkdir(dir1, FsPermission.getDirDefault()); + fs.getClient().setErasureCodingPolicy(dir1.toString(), null); + fs.mkdir(dir2, FsPermission.getDirDefault()); try { - fs.getClient().createErasureCodingZone(notEmpty.toString(), null); - fail("Erasure coding zone on non-empty dir"); + fs.getClient().setErasureCodingPolicy(dir2.toString(), null); + fail("Nested erasure coding policies"); } catch (IOException e) { - assertExceptionContains("erasure coding zone for a non-empty directory", e); + assertExceptionContains("already has an erasure coding policy", e); } - /* Verify that nested EC zones cannot be created */ - final Path zone1 = new Path("/zone1"); - final Path zone2 = new Path(zone1, "zone2"); - fs.mkdir(zone1, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(zone1.toString(), null); - fs.mkdir(zone2, FsPermission.getDirDefault()); - try { - fs.getClient().createErasureCodingZone(zone2.toString(), null); - fail("Nested erasure coding zones"); - } catch (IOException e) { - assertExceptionContains("already in an erasure coding zone", e); - } - - /* Verify that EC zone cannot be created on a file */ + /* Verify that EC policy cannot be set on a file */ final Path fPath = new Path("/file"); fs.create(fPath); try { - fs.getClient().createErasureCodingZone(fPath.toString(), null); - fail("Erasure coding zone on file"); + fs.getClient().setErasureCodingPolicy(fPath.toString(), null); + fail("Erasure coding policy on file"); } catch (IOException e) { - assertExceptionContains("erasure coding zone for a file", e); + assertExceptionContains("erasure coding policy for a file", e); } } @@ -114,8 +119,8 @@ public void testMoveValidity() throws IOException, InterruptedException { final Path dstECDir = new Path("/dstEC"); fs.mkdir(srcECDir, FsPermission.getDirDefault()); fs.mkdir(dstECDir, FsPermission.getDirDefault()); - fs.getClient().createErasureCodingZone(srcECDir.toString(), null); - fs.getClient().createErasureCodingZone(dstECDir.toString(), null); + fs.getClient().setErasureCodingPolicy(srcECDir.toString(), null); + fs.getClient().setErasureCodingPolicy(dstECDir.toString(), null); final Path srcFile = new Path(srcECDir, "foo"); fs.create(srcFile); @@ -130,37 +135,26 @@ public void testMoveValidity() throws IOException, InterruptedException { fs.rename(new Path("/dstEC/srcEC"), srcECDir); // move back // Test move file - /* Verify that a file can be moved between 2 EC zones */ + /* Verify that a file can be moved between 2 EC dirs */ fs.rename(srcFile, dstECDir); fs.rename(new Path(dstECDir, "foo"), srcECDir); // move back - /* Verify that a file cannot be moved from a non-EC dir to an EC zone */ + /* Verify that a file can be moved from a non-EC dir to an EC dir */ final Path nonECDir = new Path("/nonEC"); fs.mkdir(nonECDir, FsPermission.getDirDefault()); - try { - fs.rename(srcFile, nonECDir); - fail("A file shouldn't be able to move from a non-EC dir to an EC zone"); - } catch (IOException e) { - assertExceptionContains("can't be moved because the source and " + - "destination have different erasure coding policies", e); - } + fs.rename(srcFile, nonECDir); - /* Verify that a file cannot be moved from an EC zone to a non-EC dir */ + /* Verify that a file can be moved from an EC dir to a non-EC dir */ final Path nonECFile = new Path(nonECDir, "nonECFile"); fs.create(nonECFile); - try { - fs.rename(nonECFile, dstECDir); - } catch (IOException e) { - assertExceptionContains("can't be moved because the source and " + - "destination have different erasure coding policies", e); - } + fs.rename(nonECFile, dstECDir); } @Test public void testReplication() throws IOException { final Path testDir = new Path("/ec"); fs.mkdir(testDir, FsPermission.getDirDefault()); - fs.createErasureCodingZone(testDir, null); + fs.setErasureCodingPolicy(testDir, null); final Path fooFile = new Path(testDir, "foo"); // create ec file with replication=0 fs.create(fooFile, FsPermission.getFileDefault(), true, @@ -171,23 +165,23 @@ public void testReplication() throws IOException { } @Test - public void testGetErasureCodingInfoWithSystemDefaultECPolicy() throws Exception { + public void testGetErasureCodingPolicyWithSystemDefaultECPolicy() throws Exception { String src = "/ec"; final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); - // dir ECInfo before creating ec zone + // dir EC policy should be null assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy()); - // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, null); //Default one will be used. + // dir EC policy after setting + fs.getClient().setErasureCodingPolicy(src, null); //Default one will be used. ErasureCodingPolicy sysDefaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); verifyErasureCodingInfo(src, sysDefaultECPolicy); fs.create(new Path(ecDir, "child1")).close(); - // verify for the files in ec zone + // verify for the files in ec dir verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy); } @Test - public void testGetErasureCodingInfo() throws Exception { + public void testGetErasureCodingPolicy() throws Exception { ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices(); assertTrue("System ecPolicies should be of only 1 for now", sysECPolicies.length == 1); @@ -196,13 +190,13 @@ public void testGetErasureCodingInfo() throws Exception { String src = "/ec2"; final Path ecDir = new Path(src); fs.mkdir(ecDir, FsPermission.getDirDefault()); - // dir ECInfo before creating ec zone + // dir ECInfo before being set assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy()); - // dir ECInfo after creating ec zone - fs.getClient().createErasureCodingZone(src, usingECPolicy); + // dir ECInfo after set + fs.getClient().setErasureCodingPolicy(src, usingECPolicy); verifyErasureCodingInfo(src, usingECPolicy); fs.create(new Path(ecDir, "child1")).close(); - // verify for the files in ec zone + // verify for the files in ec dir verifyErasureCodingInfo(src + "/child1", usingECPolicy); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java index 4610ced546040..64063d20eb09b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java @@ -38,24 +38,24 @@ public void after() { @Test public void testFileStatusWithECPolicy() throws Exception { - // test directory not in EC zone + // test directory doesn't have an EC policy final Path dir = new Path("/foo"); assertTrue(fs.mkdir(dir, FsPermission.getDirDefault())); assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy()); - // test file not in EC zone + // test file doesn't have an EC policy final Path file = new Path(dir, "foo"); fs.create(file).close(); assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy()); fs.delete(file, true); final ErasureCodingPolicy ecPolicy1 = ErasureCodingPolicyManager.getSystemDefaultPolicy(); - // create EC zone on dir - fs.createErasureCodingZone(dir, ecPolicy1); + // set EC policy on dir + fs.setErasureCodingPolicy(dir, ecPolicy1); final ErasureCodingPolicy ecPolicy2 = client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy(); assertNotNull(ecPolicy2); assertTrue(ecPolicy1.equals(ecPolicy2)); - // test file in EC zone + // test file doesn't have an EC policy fs.create(file).close(); final ErasureCodingPolicy ecPolicy3 = fs.getClient().getFileInfo(file.toUri().getPath()).getErasureCodingPolicy(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 9048fbdfddb35..68cd25edeac15 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -68,7 +68,7 @@ public void setup() throws IOException { conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()) .numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java index 08a5f58591091..ca376afa4a4db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java @@ -53,7 +53,7 @@ public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index 7a0851f462022..b58184552ab51 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -78,7 +78,7 @@ public void setup() throws IOException { cluster.waitActive(); fs = cluster.getFileSystem(); - fs.getClient().createErasureCodingZone("/", null); + fs.getClient().setErasureCodingPolicy("/", null); List datanodes = cluster.getDataNodes(); for (int i = 0; i < dnNum; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java index f577ddba2806b..318eb9fee7620 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java @@ -54,7 +54,7 @@ public void setup() throws IOException { conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); cluster.waitActive(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index b383c85889c6b..5381fcabeab9e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -57,7 +57,7 @@ public class TestWriteReadStripedFile { public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); fs = cluster.getFileSystem(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java index deffbcc6a766e..50f98a3d1f0d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -30,7 +30,6 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks; @@ -48,7 +47,7 @@ public class TestWriteStripedFileWithFailure { public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); fs = cluster.getFileSystem(); } @@ -158,4 +157,4 @@ void write(FSDataOutputStream out, int i) throws IOException { throw new IOException("Failed at i=" + i, e); } } -} \ No newline at end of file +} 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 eb24fb0f9c6d9..21e60c8f62ba5 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 @@ -82,7 +82,6 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Tool; @@ -1702,7 +1701,7 @@ private void doTestBalancerWithStripedFile(Configuration conf) throws Exception cluster.waitActive(); client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy(); - client.createErasureCodingZone("/", null); + client.setErasureCodingPolicy("/", null); long totalCapacity = sum(capacities); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java index 2202b342dcd8b..9dc537c4b6999 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java @@ -56,7 +56,7 @@ public void testRead() throws Exception { conf = getConf(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient() - .createErasureCodingZone("/", null); + .setErasureCodingPolicy("/", null); try { cluster.waitActive(); doTestRead(conf, cluster, true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java index 2e084fca0b1ed..9f4c47df858ad 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java @@ -71,7 +71,7 @@ public class TestSequentialBlockGroupId { private MiniDFSCluster cluster; private FileSystem fs; private SequentialBlockGroupIdGenerator blockGrpIdGenerator; - private Path eczone = new Path("/eczone"); + private Path ecDir = new Path("/ecDir"); @Before public void setup() throws Exception { @@ -84,9 +84,9 @@ public void setup() throws Exception { fs = cluster.getFileSystem(); blockGrpIdGenerator = cluster.getNamesystem().getBlockIdManager() .getBlockGroupIdGenerator(); - fs.mkdirs(eczone); + fs.mkdirs(ecDir); cluster.getFileSystem().getClient() - .createErasureCodingZone("/eczone", null); + .setErasureCodingPolicy("/ecDir", null); } @After @@ -104,7 +104,7 @@ public void testBlockGroupIdGeneration() throws IOException { long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); // Create a file that is 4 blocks long. - Path path = new Path(eczone, "testBlockGrpIdGeneration.dat"); + Path path = new Path(ecDir, "testBlockGrpIdGeneration.dat"); DFSTestUtil.createFile(fs, path, cellSize, fileLen, blockSize, REPLICATION, SEED); List blocks = DFSTestUtil.getAllBlocks(fs, path); @@ -134,7 +134,7 @@ public void testTriggerBlockGroupIdCollision() throws IOException { // Create a file with a few blocks to rev up the global block ID // counter. - Path path1 = new Path(eczone, "testBlockGrpIdCollisionDetection_file1.dat"); + Path path1 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file1.dat"); DFSTestUtil.createFile(fs, path1, cellSize, fileLen, blockSize, REPLICATION, SEED); List blocks1 = DFSTestUtil.getAllBlocks(fs, path1); @@ -145,7 +145,7 @@ public void testTriggerBlockGroupIdCollision() throws IOException { blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); // Trigger collisions by creating a new file. - Path path2 = new Path(eczone, "testBlockGrpIdCollisionDetection_file2.dat"); + Path path2 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file2.dat"); DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize, REPLICATION, SEED); List blocks2 = DFSTestUtil.getAllBlocks(fs, path2); @@ -204,7 +204,7 @@ public long nextValue() { // Reset back to the initial value to trigger collision blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); // Trigger collisions by creating a new file. - Path path2 = new Path(eczone, "testCollisionWithLegacyBlock_file2.dat"); + Path path2 = new Path(ecDir, "testCollisionWithLegacyBlock_file2.dat"); DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize, REPLICATION, SEED); List blocks2 = DFSTestUtil.getAllBlocks(fs, path2); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index 2598fa832b252..7794f04ecd5b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -470,8 +470,8 @@ public void testMoverWithStripedFile() throws Exception { client.mkdirs(barDir, new FsPermission((short) 777), true); client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME); - // set "/bar" directory with EC zone. - client.createErasureCodingZone(barDir, null); + // set an EC policy on "/bar" directory + client.setErasureCodingPolicy(barDir, null); // write file to barDir final String fooFile = "/bar/foo"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java index 7d06a9b9c68e4..ae2cbab1688de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.junit.After; @@ -39,7 +38,6 @@ import java.io.IOException; import java.util.Arrays; -import java.util.HashSet; import java.util.List; import static org.junit.Assert.assertEquals; @@ -70,7 +68,7 @@ public void setup() throws IOException { cluster.waitActive(); fs = cluster.getFileSystem(); fs.mkdirs(dirPath); - fs.getClient().createErasureCodingZone(dirPath.toString(), null); + fs.getClient().setErasureCodingPolicy(dirPath.toString(), null); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index c472cd800d806..4351fb9f3ab30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -74,7 +74,7 @@ public void setup() throws IOException { .numDataNodes(GROUP_SIZE).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); - dfs.getClient().createErasureCodingZone("/", null); + dfs.getClient().setErasureCodingPolicy("/", null); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 694411fca103c..40572f374b92c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -452,7 +452,7 @@ public void testAddNewStripedBlock() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir, null); + fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null); // Create a file with striped block Path p = new Path(testFilePath); @@ -524,7 +524,7 @@ public void testUpdateStripedBlocks() throws IOException{ //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); - fs.getClient().getNamenode().createErasureCodingZone(testDir, null); + fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null); //create a file with striped blocks Path p = new Path(testFilePath); 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 d3689f3d9a7c8..7565e86d3d149 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 @@ -141,7 +141,7 @@ private void testPersistHelper(Configuration conf) throws IOException { private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf, boolean isUC) throws IOException{ // contruct a INode with StripedBlock for saving and loading - fsn.createErasureCodingZone("/", null, false); + fsn.setErasureCodingPolicy("/", null, false); long id = 123456789; byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes(); PermissionStatus permissionStatus = new PermissionStatus("testuser_a", @@ -425,7 +425,7 @@ public void testSupportBlockGroup() throws IOException { .build(); cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); - fs.getClient().getNamenode().createErasureCodingZone("/", null); + fs.getClient().getNamenode().setErasureCodingPolicy("/", null); Path file = new Path("/striped"); FSDataOutputStream out = fs.create(file); byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java index de299970eccd1..efa5027fc61fb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.io.IOUtils; import org.junit.After; import org.junit.Assert; @@ -66,7 +65,7 @@ public void setUp() throws IOException { dfs = cluster.getFileSystem(); dfs.mkdirs(ecDir); - dfs.getClient().createErasureCodingZone(ecDir.toString(), ecPolicy); + dfs.getClient().setErasureCodingPolicy(ecDir.toString(), ecPolicy); dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA); dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA); dfs.setStoragePolicy(ecDir, HdfsConstants.HOT_STORAGE_POLICY_NAME); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index 48b22c064d750..2a593d5b0ebbf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -217,8 +217,8 @@ public void testDeleteOp() throws Exception { try { final int len = 1024; final Path parentDir = new Path("/parentDir"); - final Path zone = new Path(parentDir, "zone"); - final Path zoneFile = new Path(zone, "zoneFile"); + final Path ecDir = new Path(parentDir, "ecDir"); + final Path ecFile = new Path(ecDir, "ecFile"); final Path contiguousFile = new Path(parentDir, "someFile"); final DistributedFileSystem dfs; final Configuration conf = new Configuration(); @@ -232,18 +232,18 @@ public void testDeleteOp() throws Exception { FSNamesystem fsn = cluster.getNamesystem(); dfs = cluster.getFileSystem(); - dfs.mkdirs(zone); + dfs.mkdirs(ecDir); - // create erasure zone - dfs.createErasureCodingZone(zone, null); - DFSTestUtil.createFile(dfs, zoneFile, len, (short) 1, 0xFEED); + // set erasure coding policy + dfs.setErasureCodingPolicy(ecDir, null); + DFSTestUtil.createFile(dfs, ecFile, len, (short) 1, 0xFEED); DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED); final FSDirectory fsd = fsn.getFSDirectory(); // Case-1: Verify the behavior of striped blocks // Get blocks of striped file - INode inodeStriped = fsd.getINode("/parentDir/zone/zoneFile"); - assertTrue("Failed to get INodeFile for /parentDir/zone/zoneFile", + INode inodeStriped = fsd.getINode("/parentDir/ecDir/ecFile"); + assertTrue("Failed to get INodeFile for /parentDir/ecDir/ecFile", inodeStriped instanceof INodeFile); INodeFile inodeStripedFile = (INodeFile) inodeStriped; BlockInfo[] stripedBlks = inodeStripedFile.getBlocks(); @@ -252,8 +252,8 @@ public void testDeleteOp() throws Exception { blockInfo.isDeleted()); } - // delete erasure zone directory - dfs.delete(zone, true); + // delete directory with erasure coding policy + dfs.delete(ecDir, true); for (BlockInfo blockInfo : stripedBlks) { assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java index 7bfaab67228d0..1e2326a587317 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java @@ -60,7 +60,7 @@ public static void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.waitActive(); - cluster.getFileSystem().getClient().createErasureCodingZone("/", null); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); fs = cluster.getFileSystem(); Path eczone = new Path("/eczone"); fs.mkdirs(eczone); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml index 5e6065843f89c..3a10a506e759e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -48,39 +48,39 @@ - help: createZone command + help: setPolicy command - -fs NAMENODE -help createZone + -fs NAMENODE -help setPolicy RegexpComparator - ^[ \t]*Create a zone to encode files using a specified policy( )* + ^[ \t]*Set a specified erasure coding policy to a directory( )* RegexpComparator - ^-createZone \[-s <policyName>\] <path>(.)* + ^-setPolicy \[-s <policyName>\] <path>(.)* - help: getZone command + help: getPolicy command - -fs NAMENODE -help getZone + -fs NAMENODE -help getPolicy SubstringComparator - Get information about the EC zone at specified path + Get erasure coding policy information about at specified path RegexpComparator - ^-getZone <path>(.)* + ^-getPolicy <path>(.)* @@ -106,63 +106,63 @@ - createZone : create a zone to encode files + setPolicy : set erasure coding policy on a directory to encode files - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s RS-6-3-64k /eczone + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir SubstringComparator - EC Zone created successfully at NAMENODE/eczone + EC policy set successfully at NAMENODE/ecdir - createZone : create a zone twice + setPolicy : set a policy twice - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone /eczone - -fs NAMENODE -createZone /eczone + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy /ecdir + -fs NAMENODE -setPolicy /ecdir - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir SubstringComparator - Directory /eczone is already in an erasure coding zone + Directory /ecdir already has an erasure coding policy - createZone : default policy + setPolicy : default policy - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone /eczone - -fs NAMENODE -getZone /eczone + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy /ecdir + -fs NAMENODE -getPolicy /ecdir - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir SubstringComparator - Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k + ErasureCodingPolicy=[Name=RS-6-3-64k - getZone : get information about the EC zone at specified path not in zone + getPolicy : get EC policy information at specified path, which doesn't have an EC policy -fs NAMENODE -mkdir /noec - -fs NAMENODE -getZone /noec + -fs NAMENODE -getPolicy /noec -fs NAMENODE -rmdir /noec @@ -170,45 +170,45 @@ SubstringComparator - Path NAMENODE/noec is not in EC zone + Path NAMENODE/noec is not erasure coded - getZone : get information about the EC zone at specified path + getPolicy : get EC policy information at specified path, which doesn't have an EC policy - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s RS-6-3-64k /eczone - -fs NAMENODE -getZone /eczone + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir + -fs NAMENODE -getPolicy /ecdir - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir SubstringComparator - Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k + ErasureCodingPolicy=[Name=RS-6-3-64k - getZone : get EC zone at specified file path + getPolicy : get EC policy information at specified path, which doesn't have an EC policy - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s RS-6-3-64k /eczone - -fs NAMENODE -touchz /eczone/ecfile - -fs NAMENODE -getZone /eczone/ecfile + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir + -fs NAMENODE -touchz /ecdir/ecfile + -fs NAMENODE -getPolicy /ecdir/ecfile - -fs NAMENODE -rm /eczone/ecfile - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rm /ecdir/ecfile + -fs NAMENODE -rmdir /ecdir SubstringComparator - Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k + ErasureCodingPolicy=[Name=RS-6-3-64k @@ -230,64 +230,64 @@ - createZone : illegal parameters - path is missing + setPolicy : illegal parameters - path is missing - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir RegexpComparator - ^-createZone: <path> is missing(.)* + ^-setPolicy: <path> is missing(.)* - createZone : illegal parameters - policy name is missing + setPolicy : illegal parameters - policy name is missing - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -s - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir RegexpComparator - ^-createZone: option -s requires 1 argument(.)* + ^-setPolicy: option -s requires 1 argument(.)* - createZone : illegal parameters - too many arguments + setPolicy : illegal parameters - too many arguments - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone /eczone1 /eczone2 + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy /ecdir1 /ecdir2 - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir SubstringComparator - -createZone: Too many arguments + -setPolicy: Too many arguments - createZone : illegal parameters - invalidpolicy + setPolicy : illegal parameters - invalidpolicy - -fs NAMENODE -mkdir /eczone - -fs NAMENODE -createZone -s invalidpolicy /eczone + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -s invalidpolicy /ecdir - -fs NAMENODE -rmdir /eczone + -fs NAMENODE -rmdir /ecdir @@ -298,62 +298,62 @@ - createZone : illegal parameters - no such file + setPolicy : illegal parameters - no such file - -fs NAMENODE -createZone /eczone + -fs NAMENODE -setPolicy /ecdir RegexpComparator - ^createZone: `/eczone': No such file or directory(.)* + ^setPolicy: `/ecdir': No such file or directory(.)* - getZone : illegal parameters - path is missing + getPolicy : illegal parameters - path is missing - -fs NAMENODE -getZone + -fs NAMENODE -getPolicy RegexpComparator - ^-getZone: <path> is missing(.)* + ^-getPolicy: <path> is missing(.)* - getZone : illegal parameters - too many arguments + getPolicy : illegal parameters - too many arguments - -fs NAMENODE -getZone /eczone /eczone + -fs NAMENODE -getPolicy /ecdir /ecdir - -fs NAMENODE -rm /eczone + -fs NAMENODE -rm /ecdir SubstringComparator - -getZone: Too many arguments + -getPolicy: Too many arguments - getZone : illegal parameters - no such file + getPolicy : illegal parameters - no such file - -fs NAMENODE -getZone /eczone + -fs NAMENODE -getPolicy /ecdir RegexpComparator - ^getZone: `/eczone': No such file or directory(.)* + ^getPolicy: `/ecdir': No such file or directory(.)* @@ -361,7 +361,7 @@ listPolicies : illegal parameters - too many parameters - -fs NAMENODE -listPolicies /eczone + -fs NAMENODE -listPolicies /ecdir From 96d6b516b2468fce346490e0b95931d1759b3d33 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 10 Sep 2015 16:31:37 -0700 Subject: [PATCH 200/212] HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC policy. Contributed by J.Andreina. Change-Id: I9211d9728480225a407d82e6c0bea1a928adfa11 --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../server/namenode/FSDirErasureCodingOp.java | 22 ++++++++++++++++++ .../hdfs/TestErasureCodingPolicies.java | 23 +++++++++++++++++++ 3 files changed, 48 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 2f133103e635a..f49a97494c0fa 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -415,3 +415,6 @@ HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and eliminate notion of EC zones. (zhz) + + HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC + policy. (andreina via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java index 4162760d2a59b..d39da28d988ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java @@ -28,6 +28,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; + +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.permission.FsAction; @@ -105,6 +107,26 @@ static List createErasureCodingPolicyXAttr(final FSNamesystem fsn, // System default erasure coding policy will be used since no specified. if (ecPolicy == null) { ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + } else { + // If ecPolicy is specified check if it is one among active policies. + boolean validPolicy = false; + ErasureCodingPolicy[] activePolicies = + FSDirErasureCodingOp.getErasureCodingPolicies(fsd.getFSNamesystem()); + for (ErasureCodingPolicy activePolicy : activePolicies) { + if (activePolicy.equals(ecPolicy)) { + validPolicy = true; + break; + } + } + if (!validPolicy) { + List ecPolicyNames = new ArrayList(); + for (ErasureCodingPolicy activePolicy : activePolicies) { + ecPolicyNames.add(activePolicy.getName()); + } + throw new HadoopIllegalArgumentException("Policy [ " + + ecPolicy.getName()+ " ] does not match any of the " + + "supported policies. Please select any one of " + ecPolicyNames); + } } final XAttr ecXAttr; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java index f60d77d8bd5dd..ed41f7aee67a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -208,4 +209,26 @@ private void verifyErasureCodingInfo( assertEquals("Actually used ecPolicy should be equal with target ecPolicy", usingECPolicy, ecPolicy); } + + @Test + public void testCreationErasureCodingZoneWithInvalidPolicy() + throws IOException { + ECSchema rsSchema = new ECSchema("rs", 4, 2); + String policyName = "RS-4-2-128k"; + int cellSize = 128 * 1024; + ErasureCodingPolicy ecPolicy= + new ErasureCodingPolicy(policyName,rsSchema,cellSize); + String src = "/ecZone4-2"; + final Path ecDir = new Path(src); + try { + fs.mkdir(ecDir, FsPermission.getDirDefault()); + fs.getClient().setErasureCodingPolicy(src, ecPolicy); + fail("HadoopIllegalArgumentException should be thrown for" + + "setting an invalid erasure coding policy"); + } catch (Exception e) { + assertExceptionContains("Policy [ RS-4-2-128k ] does not match " + + "any of the supported policies",e); + } + } + } From ce02b5532c3d506f8eee7af268216804815fa055 Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Sat, 12 Sep 2015 13:29:25 +0530 Subject: [PATCH 201/212] HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile (Contributed by J.Andreina) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 4 ++++ .../org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index f49a97494c0fa..47bab0b6e1efb 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -418,3 +418,7 @@ HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC policy. (andreina via zhz) + + HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile + (J.Andreina via vinayakumarb) + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java index 318eb9fee7620..9ab0834a61488 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java @@ -53,6 +53,7 @@ public class TestSafeModeWithStripedFile { public void setup() throws IOException { conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 100); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); cluster.waitActive(); @@ -124,6 +125,7 @@ private void doTest(int smallSize, int minStorages) throws IOException { // so the safe blocks count doesn't increment. for (int i = 0; i < minStorages - 1; i++) { cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); cluster.triggerBlockReports(); assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn)); } @@ -131,17 +133,20 @@ private void doTest(int smallSize, int minStorages) throws IOException { // the block of smallFile reaches minStorages, // so the safe blocks count increment. cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); cluster.triggerBlockReports(); assertEquals(1, NameNodeAdapter.getSafeModeSafeBlocks(nn)); // the 2 blocks of bigFile need DATA_BLK_NUM storages to be safe for (int i = minStorages; i < DATA_BLK_NUM - 1; i++) { cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); cluster.triggerBlockReports(); assertTrue(nn.isInSafeMode()); } cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); cluster.triggerBlockReports(); assertFalse(nn.isInSafeMode()); } From c2ebab67acf421f52dfa4879261982fed2768c82 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Tue, 15 Sep 2015 10:16:45 -0700 Subject: [PATCH 202/212] HDFS-7351. Document the HDFS Erasure Coding feature. Contributed by Uma Maheswara Rao G and Zhe Zhang. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../src/site/markdown/HDFSCommands.md | 20 +++ .../src/site/markdown/HDFSErasureCoding.md | 148 ++++++++++++++++++ hadoop-project/src/site/site.xml | 1 + 4 files changed, 171 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 47bab0b6e1efb..39b5adc6a86a3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -422,3 +422,5 @@ HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile (J.Andreina via vinayakumarb) + HDFS-7351. Document the HDFS Erasure Coding feature. + (umamahesh and Zhe Zhang via wang) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md index 1c2026c216b11..c7794c626615b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md @@ -36,6 +36,7 @@ HDFS Commands Guide * [crypto](#crypto) * [datanode](#datanode) * [dfsadmin](#dfsadmin) + * [erasurecode](#erasurecode) * [haadmin](#haadmin) * [journalnode](#journalnode) * [mover](#mover) @@ -379,6 +380,25 @@ Usage: Runs a HDFS dfsadmin client. +### `erasurecode` + +Usage: + + hdfs erasurecode [generic options] + [-setPolicy [-s ] ] + [-getPolicy ] + [-listPolicies] + [-usage [cmd ...]] + [-help [cmd ...]] + +| COMMAND\_OPTION | Description | +|:---- |:---- | +|-setPolicy| Set a specified ErasureCoding policy to a directory| +|-getPolicy| Get ErasureCoding policy information about a specified path| +|-listPolicies| Lists all supported ErasureCoding policies| + +Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Administrative_commands) for more information on this command. + ### `haadmin` Usage: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md new file mode 100644 index 0000000000000..44c209e961242 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md @@ -0,0 +1,148 @@ + + +HDFS Erasure Coding +=================== + +* [HDFS Erasure Coding](#HDFS_Erasure_Coding) + * [Purpose](#Purpose) + * [Background](#Background) + * [Architecture](#Architecture) + * [Hardware resources](#Hardware_resources) + * [Deployment](#Deployment) + * [Configuration details](#Configuration_details) + * [Deployment details](#Deployment_details) + * [Administrative commands](#Administrative_commands) + +Purpose +------- + Replication is expensive -- the default 3x replication scheme has 200% overhead in storage space and other resources (e.g., network bandwidth). + However, for “warm” and “cold” datasets with relatively low I/O activities, secondary block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the primary ones. + + Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault tolerance with much less storage space. In typical Erasure Coding(EC) setups, the storage overhead is ≤ 50%. + +Background +---------- + + In storage systems, the most notable usage of EC is Redundant Array of Inexpensive Disks (RAID). RAID implements EC through striping, which divides logically sequential data (such as a file) into smaller units (such as bit, byte, or block) and stores consecutive units on different disks. In the rest of this guide this unit of striping distribution is termed a striping cell (or cell). For each stripe of original data cells, a certain number of parity cells are calculated and stored -- the process of which is called encoding. The error on any striping cell can be recovered through decoding calculation based on surviving data and parity cells. + + Integrating the EC function with HDFS could get storage efficient deployments. It can provide similar data tolerance as traditional HDFS replication based deployments but it stores only one original replica data and parity cells. + In a typical case, A file with 6 blocks will actually be consume space of 6*3 = 18 blocks with replication factor 3. But with EC (6 data,3 parity) deployment, it will only consume space of 9 blocks. + +Architecture +------------ + In the context of EC, striping has several critical advantages. First, it enables online EC which bypasses the conversion phase and immediately saves storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking . Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces. + + As in general HDFS clusters, small files could account for over 3/4 of total storage consumption. So, In this first phase of erasure coding work, HDFS supports striping model. In the near future, HDFS will supports contiguous layout as second second phase work. So this guide focuses more on striping model EC. + + * **NameNode Extensions** - Under the striping layout, a HDFS file is logically composed of block groups, each of which contains a certain number of internal blocks. + To eliminate the need for NameNode to monitor all internal blocks, a new hierarchical block naming protocol is introduced, where the ID of a block group can be inferred from any of its internal blocks. This allows each block group to be managed as a new type of BlockInfo named BlockInfoStriped, which tracks its own internal blocks by attaching an index to each replica location. + + * **Client Extensions** - The basic principle behind the extensions is to allow the client node to work on multiple internal blocks in a block group in + parallel. + On the output / write path, DFSStripedOutputStream manages a set of data streamers, one for each DataNode storing an internal block in the current block group. The streamers mostly + work asynchronously. A coordinator takes charge of operations on the entire block group, including ending the current block group, allocating a new block group, and so forth. + On the input / read path, DFSStripedInputStream translates a requested logical byte range of data as ranges into internal blocks stored on DataNodes. It then issues read requests in + parallel. Upon failures, it issues additional read requests for decoding. + + * **DataNode Extensions** - ErasureCodingWorker(ECWorker) is for reconstructing erased erasure coding blocks and runs along with the Datanode process. Erased block details would have been found out by Namenode ReplicationMonitor thread and sent to Datanode via its heartbeat responses as discussed in the previous sections. For each reconstruction task, + i.e. ReconstructAndTransferBlock, it will start an internal daemon thread that performs 3 key tasks: + + _1.Read the data from source nodes:_ For reading the data blocks from different source nodes, it uses a dedicated thread pool. + The thread pool is initialized when ErasureCodingWorker initializes. Based on the EC policy, it schedules the read requests to all source targets and ensures only to read + minimum required input blocks for reconstruction. + + _2.Decode the data and generate the output data:_ Actual decoding/encoding is done by using RawErasureEncoder API currently. + All the erased data and/or parity blocks will be recovered together. + + _3.Transfer the generated data blocks to target nodes:_ Once decoding is finished, it will encapsulate the output data to packets and send them to + target Datanodes. + To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies. +* **ErasureCodingPolicy** + Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information: + _1.The ECScema: This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon). + + _2.The size of a striping cell. + + Client and Datanode uses EC codec framework directly for doing the endoing/decoding work. + + * **Erasure Codec Framework** + We support a generic EC framework which allows system users to define, configure, and deploy multiple coding schemas such as conventional Reed-Solomon, HitchHicker and + so forth. + ErasureCoder is provided to encode or decode for a block group in the middle level, and RawErasureCoder is provided to perform the concrete algorithm calculation in the low level. ErasureCoder can + combine and make use of different RawErasureCoders for tradeoff. We abstracted coder type, data blocks size, parity blocks size into ECSchema. A default system schema using RS (6, 3) is built-in. + For the system default codec Reed-Solomon we implemented both RSRawErasureCoder in pure Java and NativeRawErasureCoder based on Intel ISA-L. Below is the performance + comparing for different coding chunk size. We can see that the native coder can outperform the Java coder by up to 35X. + + _Intel® Storage Acceleration-Library(Intel® ISA-L)_ ISA-L is an Open Source Version and is a collection of low-level functions used in storage applications. + The open source version contains fast erasure codes that implement a general Reed-Solomon type encoding for blocks of data that helps protect against + erasure of whole blocks. The general ISA-L library contains an expanded set of functions used for data protection, hashing, encryption, etc. By + leveraging instruction sets like SSE, AVX and AVX2, the erasure coding functions are much optimized and outperform greatly on IA platforms. ISA-L + supports Linux, Windows and other platforms as well. Additionally, it also supports incremental coding so applications don’t have to wait all source + blocks to be available before to perform the coding, which can be used in HDFS. + +Hardware resources +------------------ + For using EC feature, you need to prepare for the following. + Depending on the ECSchemas used, we need to have minimum number of Datanodes available in the cluster. Example if we use ReedSolomon(6, 3) ECSchema, + then minimum nodes required is 9 to succeed the write. It can tolerate up to 3 failures. + +Deployment +---------- + +### Configuration details + + In the EC feature, raw coders are configurable. So, users need to decide the RawCoder algorithms. + Configure the customized algorithms with configuration key "*io.erasurecode.codecs*". + + Default Reed-Solomon based raw coders available in built, which can be configured by using the configuration key "*io.erasurecode.codec.rs.rawcoder*". + And also another default raw coder available if XOR based raw coder. Which could be configured by using "*io.erasurecode.codec.xor.rawcoder*" + + _EarasureCodingWorker Confugurations:_ + dfs.datanode.stripedread.threshold.millis - Threshold time for polling timeout for read service. Default value is 5000 + dfs.datanode.stripedread.threads – Number striped read thread pool threads. Default value is 20 + dfs.datanode.stripedread.buffer.size - Buffer size for reader service. Default value is 256 * 1024 + +### Deployment details + + With the striping model, client machine is responsible for do the EC endoing and tranferring data to the datanodes. + So, EC with striping model expects client machines with hghg end configurations especially of CPU and network. + +### Administrative commands + ErasureCoding command-line is provided to perform administrative commands related to ErasureCoding. This can be accessed by executing the following command. + + hdfs erasurecode [generic options] + [-setPolicy [-s ] ] + [-getPolicy ] + [-listPolicies] + [-usage [cmd ...]] + [-help [cmd ...]] + +Below are the details about each command. + +* **SetPolicy command**: `[-setPolicy [-s ] ]` + + SetPolicy command is used to set an ErasureCoding policy on a directory at the specified path. + + `path`: Refer to a pre-created directory in HDFS. This is a mandatory parameter. + + `policyName`: This is an optional parameter, specified using ‘-s’ flag. Refer to the name of ErasureCodingPolicy to be used for encoding files under this directory. If not specified the system default ErasureCodingPolicy will be used. + +* **GetPolicy command**: `[-getPolicy ]` + + GetPolicy command is used to get details of the ErasureCoding policy of a file or directory at the specified path. + +* **ListPolicies command**: `[-listPolicies]` + + Lists all supported ErasureCoding policies. For setPolicy command, one of these policies' name should be provided. \ No newline at end of file diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index 272ea7bbf3085..5c92d73a94af0 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -98,6 +98,7 @@ +

    From ced438a4bf50fe0ac9072c128e18249e6742956a Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Tue, 15 Sep 2015 10:43:13 -0700 Subject: [PATCH 203/212] HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode. Contributed by Rakesh R. Change-Id: I9429706ae3c9b10a9274c07b98da6ed54cce192b --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 ++- .../erasurecode/ErasureCodingWorker.java | 33 +++++++++++++++---- 3 files changed, 33 insertions(+), 7 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 39b5adc6a86a3..acf62cbe72269 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -424,3 +424,6 @@ HDFS-7351. Document the HDFS Erasure Coding feature. (umamahesh and Zhe Zhang via wang) + + HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode. + (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index c4dd496afe16d..f7cda187a621f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -400,7 +400,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size"; public static final int DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024; public static final String DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY = "dfs.datanode.stripedread.threshold.millis"; - public static final int DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s + public static final int DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s + public static final String DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size"; + public static final int DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8; public static final String DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface"; public static final String DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default"; public static final String DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index f6a5ece26557f..56b54f1590759 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -38,6 +38,7 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -92,6 +93,7 @@ public final class ErasureCodingWorker { private final DataNode datanode; private final Configuration conf; + private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL; private ThreadPoolExecutor STRIPED_READ_THREAD_POOL; private final int STRIPED_READ_THRESHOLD_MILLIS; private final int STRIPED_READ_BUFFER_SIZE; @@ -109,6 +111,10 @@ public ErasureCodingWorker(Configuration conf, DataNode datanode) { STRIPED_READ_BUFFER_SIZE = conf.getInt( DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT); + + initializeStripedBlkRecoveryThreadPool(conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT)); } private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) { @@ -142,6 +148,25 @@ public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) { STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true); } + private void initializeStripedBlkRecoveryThreadPool(int num) { + if (LOG.isDebugEnabled()) { + LOG.debug("Using striped block recovery; pool threads=" + num); + } + STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60, + TimeUnit.SECONDS, new LinkedBlockingQueue(), + new Daemon.DaemonFactory() { + private final AtomicInteger threadIdx = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable r) { + Thread t = super.newThread(r); + t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement()); + return t; + } + }); + STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true); + } + /** * Handles the Erasure Coding recovery work commands. * @@ -150,12 +175,8 @@ public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) { */ public void processErasureCodingTasks(Collection ecTasks) { for (BlockECRecoveryInfo recoveryInfo : ecTasks) { - try { - new Daemon(new ReconstructAndTransferBlock(recoveryInfo)).start(); - } catch (Throwable e) { - LOG.warn("Failed to recover striped block " + - recoveryInfo.getExtendedBlock().getLocalBlock(), e); - } + STRIPED_BLK_RECOVERY_THREAD_POOL.submit(new ReconstructAndTransferBlock( + recoveryInfo)); } } From e36129b61abd9edbdd77e053a5e2bfdad434d164 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 17 Sep 2015 09:56:32 -0700 Subject: [PATCH 204/212] HDFS-9088. Cleanup erasure coding documentation. Contributed by Andrew Wang. Change-Id: Ic3ec1f29fef0e27c46fff66fd28a51f8c4c61e71 --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 2 + .../src/site/markdown/HDFSErasureCoding.md | 123 ++++++++---------- 2 files changed, 57 insertions(+), 68 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index acf62cbe72269..0345a547913e3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -427,3 +427,5 @@ HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode. (Rakesh R via zhz) + + HDFS-9088. Cleanup erasure coding documentation. (wang via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md index 44c209e961242..3040bf52ef8bf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md @@ -19,108 +19,95 @@ HDFS Erasure Coding * [Purpose](#Purpose) * [Background](#Background) * [Architecture](#Architecture) - * [Hardware resources](#Hardware_resources) * [Deployment](#Deployment) - * [Configuration details](#Configuration_details) - * [Deployment details](#Deployment_details) + * [Cluster and hardware configuration](#Cluster_and_hardware_configuration) + * [Configuration keys](#Configuration_keys) * [Administrative commands](#Administrative_commands) Purpose ------- - Replication is expensive -- the default 3x replication scheme has 200% overhead in storage space and other resources (e.g., network bandwidth). - However, for “warm” and “cold” datasets with relatively low I/O activities, secondary block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the primary ones. + Replication is expensive -- the default 3x replication scheme in HDFS has 200% overhead in storage space and other resources (e.g., network bandwidth). + However, for warm and cold datasets with relatively low I/O activities, additional block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the first replica. - Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault tolerance with much less storage space. In typical Erasure Coding(EC) setups, the storage overhead is ≤ 50%. + Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault-tolerance with much less storage space. In typical Erasure Coding (EC) setups, the storage overhead is no more than 50%. Background ---------- In storage systems, the most notable usage of EC is Redundant Array of Inexpensive Disks (RAID). RAID implements EC through striping, which divides logically sequential data (such as a file) into smaller units (such as bit, byte, or block) and stores consecutive units on different disks. In the rest of this guide this unit of striping distribution is termed a striping cell (or cell). For each stripe of original data cells, a certain number of parity cells are calculated and stored -- the process of which is called encoding. The error on any striping cell can be recovered through decoding calculation based on surviving data and parity cells. - Integrating the EC function with HDFS could get storage efficient deployments. It can provide similar data tolerance as traditional HDFS replication based deployments but it stores only one original replica data and parity cells. - In a typical case, A file with 6 blocks will actually be consume space of 6*3 = 18 blocks with replication factor 3. But with EC (6 data,3 parity) deployment, it will only consume space of 9 blocks. + Integrating EC with HDFS can improve storage efficiency while still providing similar data durability as traditional replication-based HDFS deployments. + As an example, a 3x replicated file with 6 blocks will consume 6*3 = 18 blocks of disk space. But with EC (6 data, 3 parity) deployment, it will only consume 9 blocks of disk space. Architecture ------------ - In the context of EC, striping has several critical advantages. First, it enables online EC which bypasses the conversion phase and immediately saves storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking . Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces. + In the context of EC, striping has several critical advantages. First, it enables online EC (writing data immediately in EC format), avoiding a conversion phase and immediately saving storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking. Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces. - As in general HDFS clusters, small files could account for over 3/4 of total storage consumption. So, In this first phase of erasure coding work, HDFS supports striping model. In the near future, HDFS will supports contiguous layout as second second phase work. So this guide focuses more on striping model EC. + In typical HDFS clusters, small files can account for over 3/4 of total storage consumption. To better support small files, in this first phase of work HDFS supports EC with striping. In the future, HDFS will also support a contiguous EC layout. See the design doc and discussion on [HDFS-7285](https://issues.apache.org/jira/browse/HDFS-7285) for more information. - * **NameNode Extensions** - Under the striping layout, a HDFS file is logically composed of block groups, each of which contains a certain number of internal blocks. - To eliminate the need for NameNode to monitor all internal blocks, a new hierarchical block naming protocol is introduced, where the ID of a block group can be inferred from any of its internal blocks. This allows each block group to be managed as a new type of BlockInfo named BlockInfoStriped, which tracks its own internal blocks by attaching an index to each replica location. + * **NameNode Extensions** - Striped HDFS files are logically composed of block groups, each of which contains a certain number of internal blocks. + To reduce NameNode memory consumption from these additional blocks, a new hierarchical block naming protocol was introduced. The ID of a block group can be inferred from the ID of any of its internal blocks. This allows management at the level of the block group rather than the block. - * **Client Extensions** - The basic principle behind the extensions is to allow the client node to work on multiple internal blocks in a block group in - parallel. + * **Client Extensions** - The client read and write paths were enhanced to work on multiple internal blocks in a block group in parallel. On the output / write path, DFSStripedOutputStream manages a set of data streamers, one for each DataNode storing an internal block in the current block group. The streamers mostly work asynchronously. A coordinator takes charge of operations on the entire block group, including ending the current block group, allocating a new block group, and so forth. On the input / read path, DFSStripedInputStream translates a requested logical byte range of data as ranges into internal blocks stored on DataNodes. It then issues read requests in parallel. Upon failures, it issues additional read requests for decoding. - * **DataNode Extensions** - ErasureCodingWorker(ECWorker) is for reconstructing erased erasure coding blocks and runs along with the Datanode process. Erased block details would have been found out by Namenode ReplicationMonitor thread and sent to Datanode via its heartbeat responses as discussed in the previous sections. For each reconstruction task, - i.e. ReconstructAndTransferBlock, it will start an internal daemon thread that performs 3 key tasks: + * **DataNode Extensions** - The DataNode runs an additional ErasureCodingWorker (ECWorker) task for background recovery of failed erasure coded blocks. Failed EC blocks are detected by the NameNode, which then chooses a DataNode to do the recovery work. The recovery task is passed as a heartbeat response. This process is similar to how replicated blocks are re-replicated on failure. Reconstruction performs three key tasks: - _1.Read the data from source nodes:_ For reading the data blocks from different source nodes, it uses a dedicated thread pool. - The thread pool is initialized when ErasureCodingWorker initializes. Based on the EC policy, it schedules the read requests to all source targets and ensures only to read - minimum required input blocks for reconstruction. + 1. _Read the data from source nodes:_ Input data is read in parallel from source nodes using a dedicated thread pool. + Based on the EC policy, it schedules the read requests to all source targets and reads only the minimum number of input blocks for reconstruction. - _2.Decode the data and generate the output data:_ Actual decoding/encoding is done by using RawErasureEncoder API currently. - All the erased data and/or parity blocks will be recovered together. + 1. _Decode the data and generate the output data:_ New data and parity blocks are decoded from the input data. All missing data and parity blocks are decoded together. - _3.Transfer the generated data blocks to target nodes:_ Once decoding is finished, it will encapsulate the output data to packets and send them to - target Datanodes. - To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies. -* **ErasureCodingPolicy** - Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information: - _1.The ECScema: This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon). + 1. _Transfer the generated data blocks to target nodes:_ Once decoding is finished, the recovered blocks are transferred to target DataNodes. - _2.The size of a striping cell. + * **ErasureCoding policy** + To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies. + Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information: - Client and Datanode uses EC codec framework directly for doing the endoing/decoding work. + 1. _The ECSchema:_ This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon). - * **Erasure Codec Framework** - We support a generic EC framework which allows system users to define, configure, and deploy multiple coding schemas such as conventional Reed-Solomon, HitchHicker and - so forth. - ErasureCoder is provided to encode or decode for a block group in the middle level, and RawErasureCoder is provided to perform the concrete algorithm calculation in the low level. ErasureCoder can - combine and make use of different RawErasureCoders for tradeoff. We abstracted coder type, data blocks size, parity blocks size into ECSchema. A default system schema using RS (6, 3) is built-in. - For the system default codec Reed-Solomon we implemented both RSRawErasureCoder in pure Java and NativeRawErasureCoder based on Intel ISA-L. Below is the performance - comparing for different coding chunk size. We can see that the native coder can outperform the Java coder by up to 35X. + 1. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work. - _Intel® Storage Acceleration-Library(Intel® ISA-L)_ ISA-L is an Open Source Version and is a collection of low-level functions used in storage applications. - The open source version contains fast erasure codes that implement a general Reed-Solomon type encoding for blocks of data that helps protect against - erasure of whole blocks. The general ISA-L library contains an expanded set of functions used for data protection, hashing, encryption, etc. By - leveraging instruction sets like SSE, AVX and AVX2, the erasure coding functions are much optimized and outperform greatly on IA platforms. ISA-L - supports Linux, Windows and other platforms as well. Additionally, it also supports incremental coding so applications don’t have to wait all source - blocks to be available before to perform the coding, which can be used in HDFS. + Currently, HDFS supports the Reed-Solomon and XOR erasure coding algorithms. Additional algorithms are planned as future work. + The system default scheme is Reed-Solomon (6, 3) with a cell size of 64KB. -Hardware resources ------------------- - For using EC feature, you need to prepare for the following. - Depending on the ECSchemas used, we need to have minimum number of Datanodes available in the cluster. Example if we use ReedSolomon(6, 3) ECSchema, - then minimum nodes required is 9 to succeed the write. It can tolerate up to 3 failures. Deployment ---------- -### Configuration details +### Cluster and hardware configuration + + Erasure coding places additional demands on the cluster in terms of CPU and network. - In the EC feature, raw coders are configurable. So, users need to decide the RawCoder algorithms. - Configure the customized algorithms with configuration key "*io.erasurecode.codecs*". + Encoding and decoding work consumes additional CPU on both HDFS clients and DataNodes. - Default Reed-Solomon based raw coders available in built, which can be configured by using the configuration key "*io.erasurecode.codec.rs.rawcoder*". - And also another default raw coder available if XOR based raw coder. Which could be configured by using "*io.erasurecode.codec.xor.rawcoder*" + Erasure coded files are also spread across racks for rack fault-tolerance. + This means that when reading and writing striped files, most operations are off-rack. + Network bisection bandwidth is thus very important. - _EarasureCodingWorker Confugurations:_ - dfs.datanode.stripedread.threshold.millis - Threshold time for polling timeout for read service. Default value is 5000 - dfs.datanode.stripedread.threads – Number striped read thread pool threads. Default value is 20 - dfs.datanode.stripedread.buffer.size - Buffer size for reader service. Default value is 256 * 1024 + For rack fault-tolerance, it is also important to have at least as many racks as the configured EC stripe width. + For the default EC policy of RS (6,3), this means minimally 9 racks, and ideally 10 or 11 to handle planned and unplanned outages. + For clusters with fewer racks than the stripe width, HDFS cannot maintain rack fault-tolerance, but will still attempt + to spread a striped file across multiple nodes to preserve node-level fault-tolerance. -### Deployment details +### Configuration keys - With the striping model, client machine is responsible for do the EC endoing and tranferring data to the datanodes. - So, EC with striping model expects client machines with hghg end configurations especially of CPU and network. + The codec implementation for Reed-Solomon and XOR can be configured with the following client and DataNode configuration keys: + `io.erasurecode.codec.rs.rawcoder` and `io.erasurecode.codec.xor.rawcoder`. + The default implementations for both of these codecs are pure Java. + + Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters: + + 1. `dfs.datanode.stripedread.threshold.millis` - Timeout for striped reads. Default value is 5000 ms. + 1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads. + 1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB. ### Administrative commands - ErasureCoding command-line is provided to perform administrative commands related to ErasureCoding. This can be accessed by executing the following command. + + HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding. hdfs erasurecode [generic options] [-setPolicy [-s ] ] @@ -131,18 +118,18 @@ Deployment Below are the details about each command. -* **SetPolicy command**: `[-setPolicy [-s ] ]` + * `[-setPolicy [-s ] ]` - SetPolicy command is used to set an ErasureCoding policy on a directory at the specified path. + Sets an ErasureCoding policy on a directory at the specified path. - `path`: Refer to a pre-created directory in HDFS. This is a mandatory parameter. + `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files. - `policyName`: This is an optional parameter, specified using ‘-s’ flag. Refer to the name of ErasureCodingPolicy to be used for encoding files under this directory. If not specified the system default ErasureCodingPolicy will be used. + `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using ‘-s’ flag. If no policy is specified, the system default ErasureCodingPolicy will be used. -* **GetPolicy command**: `[-getPolicy ]` + * `[-getPolicy ]` - GetPolicy command is used to get details of the ErasureCoding policy of a file or directory at the specified path. + Get details of the ErasureCoding policy of a file or directory at the specified path. -* **ListPolicies command**: `[-listPolicies]` + * `[-listPolicies]` - Lists all supported ErasureCoding policies. For setPolicy command, one of these policies' name should be provided. \ No newline at end of file + Lists all supported ErasureCoding policies. These names are suitable for use with the `setPolicy` command. From 607bda2a21aebca7b363d57ddece8ce8b5db6c63 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 17 Sep 2015 14:52:37 -0700 Subject: [PATCH 205/212] HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when specifying policy. Contributed by Zhe Zhang. Change-Id: I82919e55c4a0d8646d25a53aac757098a56b3029 --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../hadoop/hdfs/tools/erasurecode/ECCommand.java | 6 +++--- .../src/site/markdown/HDFSErasureCoding.md | 4 ++-- .../src/test/resources/testErasureCodingConf.xml | 14 +++++++------- 4 files changed, 15 insertions(+), 12 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0345a547913e3..3a29db132c1a5 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -429,3 +429,6 @@ (Rakesh R via zhz) HDFS-9088. Cleanup erasure coding documentation. (wang via zhz) + + HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when + specifying policy. (zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java index b857d3e66353e..978fe47423ca4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -80,11 +80,11 @@ protected void processPath(PathData item) throws IOException { */ static class SetECPolicyCommand extends ECCommand { public static final String NAME = "setPolicy"; - public static final String USAGE = "[-s ] "; + public static final String USAGE = "[-p ] "; public static final String DESCRIPTION = "Set a specified erasure coding policy to a directory\n" + "Options :\n" - + " -s : erasure coding policy name to encode files. " + + " -p : erasure coding policy name to encode files. " + "If not passed the default policy will be used\n" + " : Path to a directory. Under this directory " + "files will be encoded using specified erasure coding policy"; @@ -93,7 +93,7 @@ static class SetECPolicyCommand extends ECCommand { @Override protected void processOptions(LinkedList args) throws IOException { - ecPolicyName = StringUtils.popOptionWithArgument("-s", args); + ecPolicyName = StringUtils.popOptionWithArgument("-p", args); if (args.isEmpty()) { throw new HadoopIllegalArgumentException(" is missing"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md index 3040bf52ef8bf..2323a6dcf5e3b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md @@ -110,7 +110,7 @@ Deployment HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding. hdfs erasurecode [generic options] - [-setPolicy [-s ] ] + [-setPolicy [-p ] ] [-getPolicy ] [-listPolicies] [-usage [cmd ...]] @@ -118,7 +118,7 @@ Deployment Below are the details about each command. - * `[-setPolicy [-s ] ]` + * `[-setPolicy [-p ] ]` Sets an ErasureCoding policy on a directory at the specified path. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml index 3a10a506e759e..e72d862264e0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -61,7 +61,7 @@ RegexpComparator - ^-setPolicy \[-s <policyName>\] <path>(.)* + ^-setPolicy \[-p <policyName>\] <path>(.)* @@ -109,7 +109,7 @@ setPolicy : set erasure coding policy on a directory to encode files -fs NAMENODE -mkdir /ecdir - -fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir + -fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir -fs NAMENODE -rmdir /ecdir @@ -179,7 +179,7 @@ getPolicy : get EC policy information at specified path, which doesn't have an EC policy -fs NAMENODE -mkdir /ecdir - -fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir + -fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir -fs NAMENODE -getPolicy /ecdir @@ -197,7 +197,7 @@ getPolicy : get EC policy information at specified path, which doesn't have an EC policy -fs NAMENODE -mkdir /ecdir - -fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir + -fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir -fs NAMENODE -touchz /ecdir/ecfile -fs NAMENODE -getPolicy /ecdir/ecfile @@ -250,7 +250,7 @@ setPolicy : illegal parameters - policy name is missing -fs NAMENODE -mkdir /ecdir - -fs NAMENODE -setPolicy -s + -fs NAMENODE -setPolicy -p -fs NAMENODE -rmdir /ecdir @@ -258,7 +258,7 @@ RegexpComparator - ^-setPolicy: option -s requires 1 argument(.)* + ^-setPolicy: option -p requires 1 argument(.)* @@ -284,7 +284,7 @@ setPolicy : illegal parameters - invalidpolicy -fs NAMENODE -mkdir /ecdir - -fs NAMENODE -setPolicy -s invalidpolicy /ecdir + -fs NAMENODE -setPolicy -p invalidpolicy /ecdir -fs NAMENODE -rmdir /ecdir From 82a88b92b46911c1b33d11898f8f678a134a9b69 Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 18 Sep 2015 10:45:29 -0700 Subject: [PATCH 206/212] HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to dfs.datanode.stripedread.timeout.millis. Contributed by Andrew Wang. Change-Id: Ice86c5d46d29e94087c0f186b941d5394e7ac0e6 --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 ++-- .../datanode/erasurecode/ErasureCodingWorker.java | 10 +++++----- .../org/apache/hadoop/hdfs/util/StripedBlockUtil.java | 6 +++--- .../hadoop-hdfs/src/main/resources/hdfs-default.xml | 8 ++++---- .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md | 2 +- 6 files changed, 18 insertions(+), 15 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 3a29db132c1a5..8ff696baf7e5d 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -432,3 +432,6 @@ HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when specifying policy. (zhz) + + HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to + dfs.datanode.stripedread.timeout.millis. (wang via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index f7cda187a621f..53c6cdba11db8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -399,8 +399,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20; public static final String DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size"; public static final int DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024; - public static final String DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY = "dfs.datanode.stripedread.threshold.millis"; - public static final int DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s + public static final String DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY = "dfs.datanode.stripedread.timeout.millis"; + public static final int DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT = 5000; //5s public static final String DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size"; public static final int DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8; public static final String DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index 56b54f1590759..d1fa4788d9a77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -95,16 +95,16 @@ public final class ErasureCodingWorker { private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL; private ThreadPoolExecutor STRIPED_READ_THREAD_POOL; - private final int STRIPED_READ_THRESHOLD_MILLIS; + private final int STRIPED_READ_TIMEOUT_MILLIS; private final int STRIPED_READ_BUFFER_SIZE; public ErasureCodingWorker(Configuration conf, DataNode datanode) { this.datanode = datanode; this.conf = conf; - STRIPED_READ_THRESHOLD_MILLIS = conf.getInt( - DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY, - DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT); + STRIPED_READ_TIMEOUT_MILLIS = conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT); initializeStripedReadThreadPool(conf.getInt( DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT)); @@ -556,7 +556,7 @@ private int[] readMinimumStripedData4Recovery(final int[] success) try { StripingChunkReadResult result = StripedBlockUtil.getNextCompletedStripedRead( - readService, futures, STRIPED_READ_THRESHOLD_MILLIS); + readService, futures, STRIPED_READ_TIMEOUT_MILLIS); int resultIndex = -1; if (result.state == StripingChunkReadResult.SUCCESSFUL) { resultIndex = result.index; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 4fded73c5e27c..897b092e8131b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -203,12 +203,12 @@ public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum, */ public static StripingChunkReadResult getNextCompletedStripedRead( CompletionService readService, Map, Integer> futures, - final long threshold) throws InterruptedException { + final long timeoutMillis) throws InterruptedException { Preconditions.checkArgument(!futures.isEmpty()); Future future = null; try { - if (threshold > 0) { - future = readService.poll(threshold, TimeUnit.MILLISECONDS); + if (timeoutMillis > 0) { + future = readService.poll(timeoutMillis, TimeUnit.MILLISECONDS); } else { future = readService.take(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 0db56dd3fb5e6..acbad07108bb1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -2403,23 +2403,23 @@ - dfs.datanode.stripedread.threshold.millis + dfs.datanode.stripedread.timeout.millis 5000 - datanode striped read threshold in millisecond. + Datanode striped read timeout in milliseconds. dfs.datanode.stripedread.threads 20 - datanode striped read thread pool size. + Number of threads used by the Datanode for background recovery work. dfs.datanode.stripedread.buffer.size 262144 - datanode striped read buffer size. + Datanode striped read buffer size. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md index 2323a6dcf5e3b..5b3aa34833adc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md @@ -101,7 +101,7 @@ Deployment Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters: - 1. `dfs.datanode.stripedread.threshold.millis` - Timeout for striped reads. Default value is 5000 ms. + 1. `dfs.datanode.stripedread.timeout.millis` - Timeout for striped reads. Default value is 5000 ms. 1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads. 1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB. From c457095206e5093c577b0124ad6fc7eef6f77b0a Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 18 Sep 2015 11:04:06 -0700 Subject: [PATCH 207/212] HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning. Contributed by Rakesh R. Change-Id: Ic248999a7f8e5e740d49c9b10abcf16f66dd0f98 --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../apache/hadoop/hdfs/DFSStripedInputStream.java | 14 +++++++------- .../hdfs/server/blockmanagement/BlockManager.java | 1 + .../apache/hadoop/hdfs/util/StripedBlockUtil.java | 2 +- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 8ff696baf7e5d..468cc56f51911 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -435,3 +435,6 @@ HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to dfs.datanode.stripedread.timeout.millis. (wang via zhz) + + HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning. + (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index 2ad63b88d7143..b7c22c42929c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -112,7 +112,6 @@ private static class BlockReaderInfo { * offsets for all the block readers so that we can skip data if necessary. */ long blockReaderOffset; - LocatedBlock targetBlock; /** * We use this field to indicate whether we should use this reader. In case * we hit any issue with this reader, we set this field to true and avoid @@ -120,10 +119,8 @@ private static class BlockReaderInfo { */ boolean shouldSkip = false; - BlockReaderInfo(BlockReader reader, LocatedBlock targetBlock, - DatanodeInfo dn, long offset) { + BlockReaderInfo(BlockReader reader, DatanodeInfo dn, long offset) { this.reader = reader; - this.targetBlock = targetBlock; this.datanode = dn; this.blockReaderOffset = offset; } @@ -649,8 +646,8 @@ boolean createBlockReader(LocatedBlock block, int chunkIndex) } } if (reader != null) { - readerInfos[chunkIndex] = new BlockReaderInfo(reader, block, - dnInfo.info, alignedStripe.getOffsetInBlock()); + readerInfos[chunkIndex] = new BlockReaderInfo(reader, dnInfo.info, + alignedStripe.getOffsetInBlock()); return true; } } @@ -826,7 +823,10 @@ class StatefulStripeReader extends StripeReader { void prepareDecodeInputs() { if (decodeInputs == null) { decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum]; - ByteBuffer cur = curStripeBuf.duplicate(); + final ByteBuffer cur; + synchronized (DFSStripedInputStream.this) { + cur = curStripeBuf.duplicate(); + } StripedBlockUtil.VerticalRange range = alignedStripe.range; for (int i = 0; i < dataBlkNum; i++) { cur.limit(cur.capacity()); 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 3c1c4612e3e0b..121116934b9e0 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 @@ -971,6 +971,7 @@ public void setBlockToken(final LocatedBlock b, if (isBlockTokenEnabled()) { // Use cached UGI if serving RPC calls. if (b.isStriped()) { + Preconditions.checkState(b instanceof LocatedStripedBlock); LocatedStripedBlock sb = (LocatedStripedBlock) b; int[] indices = sb.getBlockIndices(); Token[] blockTokens = new Token[indices.length]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 897b092e8131b..5af35853d215e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -372,7 +372,7 @@ public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy, // Step 4: calculate each chunk's position in destination buffer. Since the // whole read range is within a single stripe, the logic is simpler here. - int bufOffset = (int) (rangeStartInBlockGroup % (cellSize * dataBlkNum)); + int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum)); for (StripingCell cell : cells) { long cellStart = cell.idxInInternalBlk * cellSize + cell.offset; long cellEnd = cellStart + cell.size - 1; From b762199adbd10173c588df67bd227393c5bbcce9 Mon Sep 17 00:00:00 2001 From: Uma Mahesh Date: Sat, 19 Sep 2015 21:18:42 -0700 Subject: [PATCH 208/212] HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process remaining tasks due to one invalid ECTask. Contributed by Uma Maheswara Rao G. --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../erasurecode/ErasureCodingWorker.java | 9 ++- .../hadoop/hdfs/TestRecoverStripedFile.java | 62 ++++++++++++++----- 3 files changed, 57 insertions(+), 17 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 468cc56f51911..db63d533949dd 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -438,3 +438,6 @@ HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning. (Rakesh R via zhz) + + HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process + remaining tasks due to one invalid ECTask (umamahesh) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index d1fa4788d9a77..7a7cd24612c11 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -175,8 +175,13 @@ public Thread newThread(Runnable r) { */ public void processErasureCodingTasks(Collection ecTasks) { for (BlockECRecoveryInfo recoveryInfo : ecTasks) { - STRIPED_BLK_RECOVERY_THREAD_POOL.submit(new ReconstructAndTransferBlock( - recoveryInfo)); + try { + STRIPED_BLK_RECOVERY_THREAD_POOL + .submit(new ReconstructAndTransferBlock(recoveryInfo)); + } catch (Throwable e) { + LOG.warn("Failed to recover striped block " + + recoveryInfo.getExtendedBlock().getLocalBlock(), e); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index b58184552ab51..ed2ca7b367db9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -17,10 +17,23 @@ */ package org.apache.hadoop.hdfs; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -28,30 +41,20 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - 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.namenode.ErasureCodingPolicyManager; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.BitSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - public class TestRecoverStripedFile { public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class); @@ -383,4 +386,33 @@ private LocatedBlocks waitForRecoveryFinished(Path file, int groupSize) private LocatedBlocks getLocatedBlocks(Path file) throws IOException { return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE); } + + /* + * Tests that processErasureCodingTasks should not throw exceptions out due to + * invalid ECTask submission. + */ + @Test + public void testProcessErasureCodingTasksSubmitionShouldSucceed() + throws Exception { + DataNode dataNode = cluster.dataNodes.get(0).datanode; + + // Pack invalid(dummy) parameters in ecTasks. Irrespective of parameters, each task + // thread pool submission should succeed, so that it will not prevent + // processing other tasks in the list if any exceptions. + int size = cluster.dataNodes.size(); + short[] liveIndices = new short[size]; + DatanodeInfo[] dataDNs = new DatanodeInfo[size + 1]; + DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil + .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(), + new DatanodeStorage("s01")); + DatanodeStorageInfo[] dnStorageInfo = new DatanodeStorageInfo[] { + targetDnInfos_1 }; + + BlockECRecoveryInfo invalidECInfo = new BlockECRecoveryInfo( + new ExtendedBlock("bp-id", 123456), dataDNs, dnStorageInfo, liveIndices, + ErasureCodingPolicyManager.getSystemDefaultPolicy()); + List ecTasks = new ArrayList(); + ecTasks.add(invalidECInfo); + dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks); + } } From a9e6681edf17895fdb3530a8d75449b2d3207a1b Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Mon, 21 Sep 2015 13:24:02 -0700 Subject: [PATCH 209/212] HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to getAllErasureCodingPolicies. Contributed by Rakesh R. Change-Id: Id30a1ce9e2ce676d00a9220a2d3f14b8d9f00527 --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 +++ .../apache/hadoop/hdfs/DistributedFileSystem.java | 11 +++++++++++ .../hadoop/hdfs/TestErasureCodingPolicies.java | 15 +++++++++++++++ 3 files changed, 29 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index db63d533949dd..0e21d223e2cd3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -441,3 +441,6 @@ HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process remaining tasks due to one invalid ECTask (umamahesh) + + HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to + getAllErasureCodingPolicies. (Rakesh R via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 903f763d720b8..ac927ef9c02bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -2324,4 +2324,15 @@ public ErasureCodingPolicy next(final FileSystem fs, final Path p) } }.resolve(this, absF); } + + /** + * Retrieve all the erasure coding policies supported by this file system. + * + * @return all erasure coding policies supported by this file system. + * @throws IOException + */ + public Collection getAllErasureCodingPolicies() + throws IOException { + return Arrays.asList(dfs.getErasureCodingPolicies()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java index ed41f7aee67a6..0ababed63eb52 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java @@ -32,6 +32,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.Collection; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.junit.Assert.*; @@ -231,4 +232,18 @@ public void testCreationErasureCodingZoneWithInvalidPolicy() } } + @Test + public void testGetAllErasureCodingPolicies() throws Exception { + ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager + .getSystemPolices(); + assertTrue("System ecPolicies should be of only 1 for now", + sysECPolicies.length == 1); + + Collection allECPolicies = fs + .getAllErasureCodingPolicies(); + assertTrue("All ecPolicies should be of only 1 for now", + allECPolicies.size() == 1); + assertEquals("Erasure coding policy mismatches", + sysECPolicies[0], allECPolicies.iterator().next()); + } } From 7bff8ca1c872ea534a96cbbc5f70134574e289ce Mon Sep 17 00:00:00 2001 From: Kai Zheng Date: Wed, 23 Sep 2015 14:13:26 +0800 Subject: [PATCH 210/212] HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too verbose and hurt performance. Contributed by Rui Li --- .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ++ .../apache/hadoop/hdfs/DFSInputStream.java | 15 ++++++-- .../hadoop/hdfs/DFSStripedInputStream.java | 34 +++++++++++++++++++ 3 files changed, 49 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index 0e21d223e2cd3..b79ce64ac89d7 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -444,3 +444,6 @@ HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to getAllErasureCodingPolicies. (Rakesh R via zhz) + + HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too + verbose and hurt performance. (Rui Li via Kai Zheng) \ No newline at end of file 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 a5911ccd50d71..385ba4b0e04fc 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 @@ -1057,9 +1057,7 @@ protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block, } } if (chosenNode == null) { - DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() + - " after checking nodes = " + Arrays.toString(nodes) + - ", ignoredNodes = " + ignoredNodes); + reportLostBlock(block, ignoredNodes); return null; } final String dnAddr = @@ -1071,6 +1069,17 @@ protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block, return new DNAddrPair(chosenNode, targetAddr, storageType); } + /** + * Warn the user of a lost block + */ + protected void reportLostBlock(LocatedBlock lostBlock, + Collection ignoredNodes) { + DatanodeInfo[] nodes = lostBlock.getLocations(); + DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() + + " after checking nodes = " + Arrays.toString(nodes) + + ", ignoredNodes = " + ignoredNodes); + } + private static String getBestNodeDNAddrPairErrorString( DatanodeInfo nodes[], AbstractMap deadNodes, Collection ignoredNodes) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index b7c22c42929c7..131a5525cd428 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -45,8 +45,11 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.EnumSet; +import java.util.List; import java.util.Set; import java.util.Collection; import java.util.Map; @@ -154,6 +157,17 @@ void skip() { private StripeRange curStripeRange; private final CompletionService readingService; + /** + * When warning the user of a lost block in striping mode, we remember the + * dead nodes we've logged. All other striping blocks on these nodes can be + * considered lost too, and we don't want to log a warning for each of them. + * This is to prevent the log from being too verbose. Refer to HDFS-8920. + * + * To minimize the overhead, we only store the datanodeUuid in this set + */ + private final Set warnedNodes = Collections.newSetFromMap( + new ConcurrentHashMap()); + DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ErasureCodingPolicy ecPolicy, LocatedBlocks locatedBlocks) throws IOException { @@ -527,6 +541,26 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, } } + @Override + protected void reportLostBlock(LocatedBlock lostBlock, + Collection ignoredNodes) { + DatanodeInfo[] nodes = lostBlock.getLocations(); + if (nodes != null && nodes.length > 0) { + List dnUUIDs = new ArrayList<>(); + for (DatanodeInfo node : nodes) { + dnUUIDs.add(node.getDatanodeUuid()); + } + if (!warnedNodes.containsAll(dnUUIDs)) { + DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " + + "all striping blocks on them are lost. " + + "IgnoredNodes = " + ignoredNodes); + warnedNodes.addAll(dnUUIDs); + } + } else { + super.reportLostBlock(lostBlock, ignoredNodes); + } + } + /** * The reader for reading a complete {@link AlignedStripe}. Note that an * {@link AlignedStripe} may cross multiple stripes with cellSize width. From c09dc258a8f64fab852bf6f26187163480dbee3c Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Wed, 23 Sep 2015 13:34:09 -0700 Subject: [PATCH 211/212] HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from ErasureCodingPolicy. Contributed by Vinayakumar B. Change-Id: Ic56da0b426f47c63dac440aef6f5fc8554f6cf13 --- .../hadoop/hdfs/protocol/HdfsConstants.java | 11 ----- .../src/main/proto/hdfs.proto | 1 + .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 5 ++- .../hadoop/hdfs/StripedDataStreamer.java | 34 ++++++++-------- .../hadoop/hdfs/protocolPB/PBHelper.java | 3 +- .../hdfs/server/balancer/Dispatcher.java | 13 +++--- .../blockmanagement/BlockInfoStriped.java | 10 +++-- .../server/blockmanagement/BlockManager.java | 10 ++--- .../hadoop/hdfs/server/mover/Mover.java | 3 +- .../server/namenode/FSDirWriteFileOp.java | 11 +++-- .../hdfs/server/namenode/INodeFile.java | 11 +++-- .../server/protocol/BlocksWithLocations.java | 9 ++++- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 4 +- .../hadoop/hdfs/StripedFileTestUtil.java | 40 +++++++++---------- .../hdfs/TestDFSStripedInputStream.java | 7 ++-- .../hdfs/TestDFSStripedOutputStream.java | 7 ++-- ...TestDFSStripedOutputStreamWithFailure.java | 8 ++-- .../hdfs/TestReadStripedFileWithDecoding.java | 6 +-- .../TestReadStripedFileWithMissingBlocks.java | 6 +-- .../hadoop/hdfs/TestRecoverStripedFile.java | 6 +-- .../hdfs/TestSafeModeWithStripedFile.java | 6 +-- .../hadoop/hdfs/TestWriteReadStripedFile.java | 4 +- .../hdfs/TestWriteStripedFileWithFailure.java | 4 +- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 4 +- .../hdfs/server/balancer/TestBalancer.java | 7 ++-- .../blockmanagement/TestBlockInfoStriped.java | 4 +- .../TestBlockTokenWithDFSStriped.java | 8 ++-- .../TestSequentialBlockGroupId.java | 8 ++-- .../hadoop/hdfs/server/mover/TestMover.java | 7 ++-- .../TestAddOverReplicatedStripedBlocks.java | 9 +++-- .../server/namenode/TestAddStripedBlocks.java | 13 +++--- .../server/namenode/TestFSEditLogLoader.java | 9 +++-- .../hdfs/server/namenode/TestFSImage.java | 9 +++-- .../namenode/TestQuotaWithStripedBlocks.java | 5 ++- .../namenode/TestRecoverStripedBlocks.java | 8 ++-- .../server/namenode/TestStripedINodeFile.java | 5 ++- ...stOfflineImageViewerWithStripedBlocks.java | 8 ++-- .../hdfs/util/TestStripedBlockUtil.java | 10 ++--- 38 files changed, 177 insertions(+), 156 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java index a8bb7407bc0e2..0453d1d3530aa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java @@ -86,17 +86,6 @@ public final class HdfsConstants { public static final String CLIENT_NAMENODE_PROTOCOL_NAME = "org.apache.hadoop.hdfs.protocol.ClientProtocol"; - /* - * These values correspond to the values used by the system default erasure - * coding policy. - * TODO: get these values from ec policy of the associated INodeFile - */ - - public static final byte NUM_DATA_BLOCKS = 6; - public static final byte NUM_PARITY_BLOCKS = 3; - // The chunk size for striped block which is used by erasure coding - public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024; - // Timeouts for communicating with DataNode for streaming writes/reads public static final int READ_TIMEOUT = 60 * 1000; public static final int READ_TIMEOUT_EXTENSION = 5 * 1000; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto index 77c64b9f8e74a..b28ab425c8d0d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto @@ -536,6 +536,7 @@ message BlockWithLocationsProto { optional bytes indices = 5; optional uint32 dataBlockNum = 6; + optional uint32 cellSize = 7; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index b79ce64ac89d7..d62dbacd4ed80 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -446,4 +446,7 @@ getAllErasureCodingPolicies. (Rakesh R via zhz) HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too - verbose and hurt performance. (Rui Li via Kai Zheng) \ No newline at end of file + verbose and hurt performance. (Rui Li via Kai Zheng) + + HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from + ErasureCodingPolicy (Vinayakumar B via zhz) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index a2b6a676d4ef0..2f83f7c641837 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -18,25 +18,20 @@ package org.apache.hadoop.hdfs; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator; import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue; 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.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.ByteArrayManager; import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; @@ -109,6 +104,8 @@ private static void sleep(long ms, String op) throws InterruptedIOException { private final Coordinator coordinator; private final int index; private volatile boolean failed; + private final ECSchema schema; + private final int cellSize; StripedDataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, @@ -120,6 +117,8 @@ private static void sleep(long ms, String op) throws InterruptedIOException { byteArrayManage, favoredNodes); this.index = index; this.coordinator = coordinator; + this.schema = stat.getErasureCodingPolicy().getSchema(); + this.cellSize = stat.getErasureCodingPolicy().getCellSize(); } int getIndex() { @@ -135,7 +134,7 @@ boolean isFailed() { } private boolean isParityStreamer() { - return index >= NUM_DATA_BLOCKS; + return index >= schema.getNumDataUnits(); } @Override @@ -168,7 +167,7 @@ void populate() throws IOException { if (block != null) { // set numByte for the previous block group long bytes = 0; - for (int i = 0; i < NUM_DATA_BLOCKS; i++) { + for (int i = 0; i < schema.getNumDataUnits(); i++) { final ExtendedBlock b = coordinator.takeEndBlock(i); StripedBlockUtil.checkBlocks(index, block, i, b); bytes += b.getNumBytes(); @@ -183,15 +182,15 @@ void populate() throws IOException { final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock( excludedNodes); - if (lb.getLocations().length < HdfsConstants.NUM_DATA_BLOCKS) { + if (lb.getLocations().length < schema.getNumDataUnits()) { throw new IOException( "Failed to get datablocks number of nodes from namenode: blockGroupSize= " - + (HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS) + + (schema.getNumDataUnits() + schema.getNumParityUnits()) + ", blocks.length= " + lb.getLocations().length); } - final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( - (LocatedStripedBlock)lb, - BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + final LocatedBlock[] blocks = + StripedBlockUtil.parseStripedBlockGroup((LocatedStripedBlock) lb, + cellSize, schema.getNumDataUnits(), schema.getNumParityUnits()); for (int i = 0; i < blocks.length; i++) { StripedDataStreamer si = coordinator.getStripedDataStreamer(i); @@ -233,9 +232,10 @@ void populate() throws IOException { final LocatedBlock updated = callUpdateBlockForPipeline(bg); final long newGS = updated.getBlock().getGenerationStamp(); final LocatedBlock[] updatedBlks = StripedBlockUtil - .parseStripedBlockGroup((LocatedStripedBlock) updated, - BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); - for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) { + .parseStripedBlockGroup((LocatedStripedBlock) updated, cellSize, + schema.getNumDataUnits(), schema.getNumParityUnits()); + for (int i = 0; i < schema.getNumDataUnits() + + schema.getNumParityUnits(); i++) { StripedDataStreamer si = coordinator.getStripedDataStreamer(i); if (si.isFailed()) { continue; // skipping failed data streamer @@ -280,7 +280,7 @@ void populate() throws IOException { final ExtendedBlock bg = coordinator.getBlockGroup(); final ExtendedBlock newBG = newBlock(bg, newGS); - final int n = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + final int n = schema.getNumDataUnits() + schema.getNumParityUnits(); final DatanodeInfo[] newNodes = new DatanodeInfo[n]; final String[] newStorageIDs = new String[n]; for (int i = 0; i < n; i++) { 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 ece998488e349..524248cc325f0 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 @@ -192,6 +192,7 @@ public static BlockWithLocationsProto convert(BlockWithLocations blk) { StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk; builder.setIndices(PBHelperClient.getByteString(sblk.getIndices())); builder.setDataBlockNum(sblk.getDataBlockNum()); + builder.setCellSize(sblk.getCellSize()); } return builder.build(); } @@ -207,7 +208,7 @@ public static BlockWithLocations convert(BlockWithLocationsProto b) { PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size())); if (b.hasIndices()) { blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(), - (short) b.getDataBlockNum()); + (short) b.getDataBlockNum(), b.getCellSize()); } return blk; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java index 07c3c01cfc719..4c4fc275584d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java @@ -415,11 +415,14 @@ public static class DBlockStriped extends DBlock { final byte[] indices; final short dataBlockNum; + final int cellSize; - public DBlockStriped(Block block, byte[] indices, short dataBlockNum) { + public DBlockStriped(Block block, byte[] indices, short dataBlockNum, + int cellSize) { super(block); this.indices = indices; this.dataBlockNum = dataBlockNum; + this.cellSize = cellSize; } public DBlock getInternalBlock(StorageGroup storage) { @@ -429,8 +432,8 @@ public DBlock getInternalBlock(StorageGroup storage) { } byte idxInGroup = indices[idxInLocs]; long blkId = getBlock().getBlockId() + idxInGroup; - long numBytes = getInternalBlockLength(getNumBytes(), - HdfsConstants.BLOCK_STRIPED_CELL_SIZE, dataBlockNum, idxInGroup); + long numBytes = getInternalBlockLength(getNumBytes(), cellSize, + dataBlockNum, idxInGroup); Block blk = new Block(getBlock()); blk.setBlockId(blkId); blk.setNumBytes(numBytes); @@ -717,8 +720,8 @@ private long getBlockList() throws IOException { bytesReceived += sblkLocs.getBlock().getNumBytes() / sblkLocs.getDataBlockNum(); block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(), - sblkLocs.getDataBlockNum()); - } else{ + sblkLocs.getDataBlockNum(), sblkLocs.getCellSize()); + } else { bytesReceived += blkLocs.getBlock().getNumBytes(); block = new DBlock(blkLocs.getBlock()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index df48655009c18..d1067b3e037ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -22,8 +22,6 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - /** * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. * @@ -65,6 +63,10 @@ public short getParityBlockNum() { return (short) ecPolicy.getNumParityUnits(); } + public int getCellSize() { + return ecPolicy.getCellSize(); + } + /** * If the block is committed/completed and its length is less than a full * stripe, it returns the the number of actual data blocks. @@ -73,7 +75,7 @@ public short getParityBlockNum() { public short getRealDataBlockNum() { if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) { return (short) Math.min(getDataBlockNum(), - (getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1); + (getNumBytes() - 1) / ecPolicy.getCellSize() + 1); } else { return getDataBlockNum(); } @@ -200,7 +202,7 @@ public long spaceConsumed() { // `getNumBytes` is the total of actual data block size. return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(), ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(), - BLOCK_STRIPED_CELL_SIZE); + ecPolicy.getCellSize()); } @Override 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 e7f9262b2df5b..18bfc41cfbbf0 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 @@ -92,7 +92,6 @@ import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; import org.apache.hadoop.metrics2.util.MBeans; @@ -2554,10 +2553,9 @@ private BlockToMarkCorrupt checkReplicaCorrupt( BlockIdManager.convertToStripedID(reported.getBlockId()); BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock; int reportedBlkIdx = BlockIdManager.getBlockIndex(reported); - wrongSize = reported.getNumBytes() != - getInternalBlockLength(stripedBlock.getNumBytes(), - BLOCK_STRIPED_CELL_SIZE, - stripedBlock.getDataBlockNum(), reportedBlkIdx); + wrongSize = reported.getNumBytes() != getInternalBlockLength( + stripedBlock.getNumBytes(), stripedBlock.getCellSize(), + stripedBlock.getDataBlockNum(), reportedBlkIdx); } else { wrongSize = storedBlock.getNumBytes() != reported.getNumBytes(); } @@ -3413,7 +3411,7 @@ private long addBlock(BlockInfo block, List results) { (byte) blockStriped.getStorageBlockIndex(locations.get(i)); } results.add(new StripedBlockWithLocations(blkWithLocs, indices, - blockStriped.getDataBlockNum())); + blockStriped.getDataBlockNum(), blockStriped.getCellSize())); // approximate size return block.getNumBytes() / blockStriped.getDataBlockNum(); }else{ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java index 34d92d0d7f1c5..1e07cad69b6b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java @@ -185,7 +185,8 @@ DBlock newDBlock(LocatedBlock lb, List locations, for (int i = 0; i < indices.length; i++) { indices[i] = (byte) lsb.getBlockIndices()[i]; } - db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits()); + db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits(), + ecPolicy.getCellSize()); } else { db = new DBlock(blk); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 867b4515469c8..cfc939ef1d56a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -204,9 +204,14 @@ static ValidateAddBlockResult validateAddBlock( clientMachine = pendingFile.getFileUnderConstructionFeature() .getClientMachine(); isStriped = pendingFile.isStriped(); - numTargets = isStriped ? - HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : - pendingFile.getFileReplication(); + ErasureCodingPolicy ecPolicy = null; + if (isStriped) { + ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src); + numTargets = (short) (ecPolicy.getSchema().getNumDataUnits() + + ecPolicy.getSchema().getNumParityUnits()); + } else { + numTargets = pendingFile.getFileReplication(); + } storagePolicyID = pendingFile.getStoragePolicyID(); return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID, clientMachine, isStriped); 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 c765edce7141b..242c456bdb515 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 @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; @@ -388,8 +388,13 @@ public short getPreferredBlockReplication() { } max = maxInSnapshot > max ? maxInSnapshot : max; } - return isStriped() ? - HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max; + if(!isStriped()){ + return max; + } + // TODO support more policies based on policyId + ErasureCodingPolicy ecPolicy = + ErasureCodingPolicyManager.getSystemDefaultPolicy(); + return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()); } /** Set the replication factor of this file. */ 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 0507faf1af128..b6eddb61579da 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 @@ -95,16 +95,17 @@ private StringBuilder appendString(int i, StringBuilder b) { public static class StripedBlockWithLocations extends BlockWithLocations { final byte[] indices; final short dataBlockNum; + final int cellSize; public StripedBlockWithLocations(BlockWithLocations blk, byte[] indices, - short dataBlockNum) { + short dataBlockNum, int cellSize) { super(blk.getBlock(), blk.getDatanodeUuids(), blk.getStorageIDs(), blk.getStorageTypes()); Preconditions.checkArgument( blk.getDatanodeUuids().length == indices.length); this.indices = indices; this.dataBlockNum = dataBlockNum; - + this.cellSize = cellSize; } public byte[] getIndices() { @@ -114,6 +115,10 @@ public byte[] getIndices() { public short getDataBlockNum() { return dataBlockNum; } + + public int getCellSize() { + return cellSize; + } } private final BlockWithLocations[] blocks; 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 12d48114873be..274d3195fb97c 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 @@ -166,8 +166,8 @@ import org.mockito.internal.util.reflection.Whitebox; import com.google.common.annotations.VisibleForTesting; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; /** Utilities for HDFS tests */ public class DFSTestUtil { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index ded86e7fb384b..8d4a0cf5c9bf6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; 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.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.web.ByteRangeInputStream; @@ -38,18 +37,19 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; - public class StripedFileTestUtil { public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class); + /* + * These values correspond to the values used by the system default erasure + * coding policy. + */ + public static final short NUM_DATA_BLOCKS = (short) 6; + public static final short NUM_PARITY_BLOCKS = (short) 3; + public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024; - static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - - static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; static final int stripesPerBlock = 4; - static final int blockSize = cellSize * stripesPerBlock; - static final int numDNs = dataBlocks + parityBlocks + 2; + static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock; + static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2; static final Random random = new Random(); @@ -85,9 +85,9 @@ static void verifyLength(FileSystem fs, Path srcPath, int fileLength) static void verifyPread(FileSystem fs, Path srcPath, int fileLength, byte[] expected, byte[] buf) throws IOException { try (FSDataInputStream in = fs.open(srcPath)) { - int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102, - cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102, - cellSize * dataBlocks, fileLength - 102, fileLength - 1}; + int[] startOffsets = {0, 1, BLOCK_STRIPED_CELL_SIZE - 102, BLOCK_STRIPED_CELL_SIZE, BLOCK_STRIPED_CELL_SIZE + 102, + BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1), BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1) + 102, + BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, fileLength - 102, fileLength - 1}; for (int startOffset : startOffsets) { startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); int remaining = fileLength - startOffset; @@ -158,21 +158,21 @@ static void verifySeek(FileSystem fs, Path srcPath, int fileLength) pos = 0; assertSeekAndRead(in, pos, fileLength); - if (fileLength > cellSize) { + if (fileLength > BLOCK_STRIPED_CELL_SIZE) { // seek to cellSize boundary - pos = cellSize - 1; + pos = BLOCK_STRIPED_CELL_SIZE - 1; assertSeekAndRead(in, pos, fileLength); } - if (fileLength > cellSize * dataBlocks) { + if (fileLength > BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS) { // seek to striped cell group boundary - pos = cellSize * dataBlocks - 1; + pos = BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS - 1; assertSeekAndRead(in, pos, fileLength); } - if (fileLength > blockSize * dataBlocks) { + if (fileLength > blockSize * NUM_DATA_BLOCKS) { // seek to striped block group boundary - pos = blockSize * dataBlocks - 1; + pos = blockSize * NUM_DATA_BLOCKS - 1; assertSeekAndRead(in, pos, fileLength); } @@ -235,12 +235,12 @@ static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { * number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS. */ public static short getRealDataBlockNum(int numBytes) { - return (short) Math.min(dataBlocks, + return (short) Math.min(NUM_DATA_BLOCKS, (numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1); } public static short getRealTotalBlockNum(int numBytes) { - return (short) (getRealDataBlockNum(numBytes) + parityBlocks); + return (short) (getRealDataBlockNum(numBytes) + NUM_PARITY_BLOCKS); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 6fcf644dd8be8..549a744bc7de5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -24,7 +24,6 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; @@ -60,9 +59,9 @@ public class TestDFSStripedInputStream { private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); private final ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); - private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; - private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; - private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int NUM_STRIPE_PER_BLOCK = 2; private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; private final int BLOCK_GROUP_SIZE = DATA_BLK_NUM * INTERNAL_BLOCK_SIZE; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 4263ffaf28993..0641e8ea65e94 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; @@ -51,13 +50,13 @@ public class TestDFSStripedOutputStream { GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); } - private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + private int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; private MiniDFSCluster cluster; private DistributedFileSystem fs; private Configuration conf; - private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int stripesPerBlock = 4; private final int blockSize = cellSize * stripesPerBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index afb7f95cbb0bd..44a29e671e5e6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -65,9 +65,9 @@ public class TestDFSStripedOutputStreamWithFailure { GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); } - private static final int NUM_DATA_BLOCKS = HdfsConstants.NUM_DATA_BLOCKS; - private static final int NUM_PARITY_BLOCKS = HdfsConstants.NUM_PARITY_BLOCKS; - private static final int CELL_SIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static final int NUM_PARITY_BLOCKS = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private static final int CELL_SIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private static final int STRIPES_PER_BLOCK = 4; private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK; private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS; @@ -255,7 +255,7 @@ public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException { DatanodeReportType.LIVE); assertEquals("Mismatches number of live Dns ", numDatanodes, info.length); Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes"); - int fileLength = HdfsConstants.BLOCK_STRIPED_CELL_SIZE - 1000; + int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000; final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); DFSTestUtil.writeFile(dfs, srcPath, new String(expected)); StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 68cd25edeac15..9cf970b7f4d97 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -46,16 +46,16 @@ import java.nio.ByteBuffer; import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks; public class TestReadStripedFileWithDecoding { static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class); private MiniDFSCluster cluster; private DistributedFileSystem fs; + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int smallFileLength = blockSize * dataBlocks - 123; private final int largeFileLength = blockSize * dataBlocks + 123; private final int[] fileLengths = {smallFileLength, largeFileLength}; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java index ca376afa4a4db..f521d8edb1839 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java @@ -32,8 +32,6 @@ import java.io.IOException; import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; /** @@ -46,6 +44,8 @@ public class TestReadStripedFileWithMissingBlocks { private static MiniDFSCluster cluster; private static DistributedFileSystem fs; private static Configuration conf = new HdfsConfiguration(); + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int fileLength = blockSize * dataBlocks + 123; @Before @@ -112,7 +112,7 @@ private void readFileWithMissingBlocks(Path srcPath, int fileLength, } for (int i = 0; i < missingParityNum; i++) { missingDataNodes[i + missingDataNum] = i + - Math.min(StripedFileTestUtil.dataBlocks, dataBlocks); + Math.min(StripedFileTestUtil.NUM_DATA_BLOCKS, dataBlocks); } stopDataNodes(locs, missingDataNodes); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java index ed2ca7b367db9..75dc6a02ac42a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -58,9 +58,9 @@ public class TestRecoverStripedFile { public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class); - private static final int dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS; - private static final int parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS; - private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final int dataBlkNum = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static final int parityBlkNum = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private static final int blockSize = cellSize * 3; private static final int groupSize = dataBlkNum + parityBlkNum; private static final int dnNum = groupSize + parityBlkNum; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java index 9ab0834a61488..9853b8a59fd05 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java @@ -40,10 +40,10 @@ public class TestSafeModeWithStripedFile { - static final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; - static final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; + static final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + static final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; static final int numDNs = DATA_BLK_NUM + PARITY_BLK_NUM; - static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; static final int blockSize = cellSize * 2; static MiniDFSCluster cluster; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java index 5381fcabeab9e..3d51f0c725c7d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -37,8 +37,6 @@ import java.nio.ByteBuffer; import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock; @@ -46,6 +44,8 @@ public class TestWriteReadStripedFile { public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class); private static MiniDFSCluster cluster; private static DistributedFileSystem fs; + private static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private static short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; private static Configuration conf = new HdfsConfiguration(); static { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java index c28bff82d2a04..c0dca4e8662a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -31,9 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks; import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; -import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks; public class TestWriteStripedFileWithFailure { public static final Log LOG = LogFactory @@ -41,6 +39,8 @@ public class TestWriteStripedFileWithFailure { private static MiniDFSCluster cluster; private static FileSystem fs; private static Configuration conf = new HdfsConfiguration(); + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; private final int smallFileLength = blockSize * dataBlocks - 123; private final int largeFileLength = blockSize * dataBlocks + 123; private final int[] fileLengths = {smallFileLength, largeFileLength}; 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 00a4575253d8a..0d51ac899b338 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 @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -201,7 +202,8 @@ private static BlockWithLocations getBlockWithLocations( BlockWithLocations blkLocs = new BlockWithLocations(new Block(bid, 0, 1), datanodeUuids, storageIDs, storageTypes); if (isStriped) { - blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum); + blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum, + StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE); } return blkLocs; } 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 25012e7406648..124bf8060eda3 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 @@ -67,6 +67,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.NameNodeProxies; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -149,10 +150,10 @@ static void initConfWithRamDisk(Configuration conf, conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L); } - int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; int groupSize = dataBlocks + parityBlocks; - private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final static int stripesPerBlock = 4; static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java index 6cc1dcd44faae..4650dce6ed483 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -31,8 +31,8 @@ import java.io.ByteArrayOutputStream; import java.nio.ByteBuffer; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java index 9dc537c4b6999..ae33ffe3401b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java @@ -20,7 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.balancer.TestBalancer; @@ -31,9 +31,9 @@ public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS { - private final static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - private final static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final static int stripesPerBlock = 4; private final static int numDNs = dataBlocks + parityBlocks + 2; private static MiniDFSCluster cluster; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java index 9f4c47df858ad..00b4e3bba37ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.test.GenericTestUtils; @@ -58,9 +58,9 @@ public class TestSequentialBlockGroupId { private final short REPLICATION = 1; private final long SEED = 0; - private final int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - private final int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int stripesPerBlock = 2; private final int blockSize = cellSize * stripesPerBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index 7794f04ecd5b6..3a9748f223942 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.NameNodeProxies; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -415,9 +416,9 @@ public void testMoverFailedRetry() throws Exception { } } - int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; - private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final static int stripesPerBlock = 4; static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java index ae2cbab1688de..64d33a406fa82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -49,10 +50,10 @@ public class TestAddOverReplicatedStripedBlocks { private DistributedFileSystem fs; private final Path dirPath = new Path("/striped"); private Path filePath = new Path(dirPath, "file"); - private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; - private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; - private final short GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM; - private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final short GROUP_SIZE = (short) (DATA_BLK_NUM + PARITY_BLK_NUM); + private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int NUM_STRIPE_PER_BLOCK = 4; private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; private final int numDNs = GROUP_SIZE + 3; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 4351fb9f3ab30..836bff1b4a7c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.DatanodeID; @@ -57,13 +58,13 @@ import java.util.UUID; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; import static org.junit.Assert.assertEquals; public class TestAddStripedBlocks { - private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + - HdfsConstants.NUM_PARITY_BLOCKS; + private final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS + + StripedFileTestUtil.NUM_PARITY_BLOCKS); private MiniDFSCluster cluster; private DistributedFileSystem dfs; @@ -163,8 +164,8 @@ private void checkStripedBlockUC(BlockInfoStriped block, boolean checkReplica) { assertEquals(0, block.numNodes()); Assert.assertFalse(block.isComplete()); - Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum()); - Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, + Assert.assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, block.getDataBlockNum()); + Assert.assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, block.getParityBlockNum()); Assert.assertEquals(0, block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 5298980f84d9b..917b0f2e12707 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -450,8 +451,8 @@ public void testAddNewStripedBlock() throws IOException{ long blkId = 1; long blkNumBytes = 1024; long timestamp = 1426222918; - short blockNum = HdfsConstants.NUM_DATA_BLOCKS; - short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; + short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS; + short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS; //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); @@ -522,8 +523,8 @@ public void testUpdateStripedBlocks() throws IOException{ long blkId = 1; long blkNumBytes = 1024; long timestamp = 1426222918; - short blockNum = HdfsConstants.NUM_DATA_BLOCKS; - short parityNum = HdfsConstants.NUM_PARITY_BLOCKS; + short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS; + short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS; //set the storage policy of the directory fs.mkdir(new Path(testDir), new FsPermission("755")); 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 7565e86d3d149..e45d08d6c5860 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 @@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; @@ -414,8 +415,8 @@ public void testZeroBlockSize() throws Exception { */ @Test public void testSupportBlockGroup() throws IOException { - final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS + - HdfsConstants.NUM_PARITY_BLOCKS; + final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS + + StripedFileTestUtil.NUM_PARITY_BLOCKS); final int BLOCK_SIZE = 8 * 1024 * 1024; Configuration conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); @@ -447,8 +448,8 @@ public void testSupportBlockGroup() throws IOException { BlockInfo[] blks = inode.getBlocks(); assertEquals(1, blks.length); assertTrue(blks[0].isStriped()); - assertEquals(HdfsConstants.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum()); - assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum()); + assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum()); } finally { cluster.shutdown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java index efa5027fc61fb..26f9b8ee4f765 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -113,9 +114,9 @@ public void testUpdatingQuotaCount() throws Exception { final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature() .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK); // In this case the file's real size is cell size * block group size. - Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, + Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, actualSpaceUsed); - Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, + Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, actualDiskUsed); } finally { IOUtils.cleanup(null, out); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index 3134373dab4df..6774aedba3329 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -35,16 +35,16 @@ import org.junit.Test; import java.util.List; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS; -import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class TestRecoverStripedBlocks { private final short GROUP_SIZE = - NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + (short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS); private MiniDFSCluster cluster; private final Path dirPath = new Path("/dir"); private Path filePath = new Path(dirPath, "file"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java index c8275822df47a..9f7ea3e7bc02f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; @@ -222,8 +223,8 @@ public void testDeleteOp() throws Exception { final Path contiguousFile = new Path(parentDir, "someFile"); final DistributedFileSystem dfs; final Configuration conf = new Configuration(); - final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS - + HdfsConstants.NUM_PARITY_BLOCKS; + final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS + + StripedFileTestUtil.NUM_PARITY_BLOCKS); conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java index 1e2326a587317..6d4d79734e9a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.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.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; @@ -44,12 +44,12 @@ import org.junit.Test; public class TestOfflineImageViewerWithStripedBlocks { - private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS; - private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS; + private static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; private static MiniDFSCluster cluster; private static DistributedFileSystem fs; - private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private static final int stripesPerBlock = 3; private static final int blockSize = cellSize * stripesPerBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java index e61ac07e975a4..95b0135720f91 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -21,9 +21,9 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; @@ -77,10 +77,10 @@ * TODO: test parity block logic */ public class TestStripedBlockUtil { - private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS; - private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS; - private final short BLK_GROUP_WIDTH = DATA_BLK_NUM + PARITY_BLK_NUM; - private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE; + private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final short BLK_GROUP_WIDTH = (short) (DATA_BLK_NUM + PARITY_BLK_NUM); + private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE; /** number of full stripes in a full block group */ private final int BLK_GROUP_STRIPE_NUM = 16; From 6419900ac24a5493827abf9b5d90373bc1043e0b Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 28 Sep 2015 14:40:27 -0700 Subject: [PATCH 212/212] HDFS-9040. Erasure coding: coordinate data streamers in DFSStripedOutputStream. Contributed by Jing Zhao and Walter Su. --- .../hadoop/hdfs/protocol/DatanodeID.java | 2 + .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 + .../apache/hadoop/hdfs/DFSOutputStream.java | 62 +- .../hadoop/hdfs/DFSStripedOutputStream.java | 603 +++++++++++++----- .../org/apache/hadoop/hdfs/DataStreamer.java | 212 +++--- .../hadoop/hdfs/StripedDataStreamer.java | 342 +++------- .../BlockUnderConstructionFeature.java | 30 +- .../blockmanagement/DatanodeManager.java | 4 + .../hadoop/hdfs/util/StripedBlockUtil.java | 23 +- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 31 +- .../hadoop/hdfs/StripedFileTestUtil.java | 213 ++++++- .../hdfs/TestDFSStripedOutputStream.java | 144 +---- ...TestDFSStripedOutputStreamWithFailure.java | 300 ++++----- .../hdfs/TestWriteStripedFileWithFailure.java | 8 + .../hdfs/server/balancer/TestBalancer.java | 4 +- .../hadoop/hdfs/server/mover/TestMover.java | 4 +- .../TestAddOverReplicatedStripedBlocks.java | 12 +- .../namenode/ha/TestRetryCacheWithHA.java | 9 +- 18 files changed, 1068 insertions(+), 938 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java index 6d72285426106..c709cbdd47135 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java @@ -38,6 +38,8 @@ @InterfaceStability.Evolving public class DatanodeID implements Comparable { public static final DatanodeID[] EMPTY_ARRAY = {}; + public static final DatanodeID EMPTY_DATANODE_ID = new DatanodeID("null", + "null", "null", 0, 0, 0, 0); private String ipAddr; // IP address private String hostName; // hostname claimed by datanode diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt index d62dbacd4ed80..6a01d616f9503 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -450,3 +450,6 @@ HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from ErasureCodingPolicy (Vinayakumar B via zhz) + + HDFS-9040. Erasure coding: coordinate data streamers in + DFSStripedOutputStream. (jing9 and Walter Su) 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 4923c86f7f833..e77a00a8196ae 100755 --- 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 @@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException; import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException; import org.apache.hadoop.hdfs.server.namenode.SafeModeException; import org.apache.hadoop.hdfs.util.ByteArrayManager; @@ -212,14 +213,17 @@ private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress, /** Construct a new output stream for creating a file. */ protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, EnumSet flag, Progressable progress, - DataChecksum checksum, String[] favoredNodes) throws IOException { + DataChecksum checksum, String[] favoredNodes, boolean createStreamer) + throws IOException { this(dfsClient, src, progress, stat, checksum); this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK); computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum); - streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum, - cachingStrategy, byteArrayManager, favoredNodes); + if (createStreamer) { + streamer = new DataStreamer(stat, null, dfsClient, src, progress, + checksum, cachingStrategy, byteArrayManager, favoredNodes); + } } static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, @@ -276,7 +280,7 @@ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, flag, progress, checksum, favoredNodes); } else { out = new DFSOutputStream(dfsClient, src, stat, - flag, progress, checksum, favoredNodes); + flag, progress, checksum, favoredNodes, true); } out.start(); return out; @@ -476,7 +480,7 @@ protected void adjustChunkBoundary() { * * @throws IOException */ - protected void endBlock() throws IOException { + void endBlock() throws IOException { if (getStreamer().getBytesCurBlock() == blockSize) { setCurrentPacketToEmpty(); enqueueCurrentPacket(); @@ -921,4 +925,52 @@ protected DataStreamer getStreamer() { public String toString() { return getClass().getSimpleName() + ":" + streamer; } + + static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient, + String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes) + throws IOException { + final DfsClientConf conf = dfsClient.getConf(); + int retries = conf.getNumBlockWriteLocateFollowingRetry(); + long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs(); + long localstart = Time.monotonicNow(); + while (true) { + try { + return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock, + excludedNodes, fileId, favoredNodes); + } catch (RemoteException e) { + IOException ue = e.unwrapRemoteException(FileNotFoundException.class, + AccessControlException.class, + NSQuotaExceededException.class, + DSQuotaExceededException.class, + QuotaByStorageTypeExceededException.class, + UnresolvedPathException.class); + if (ue != e) { + throw ue; // no need to retry these exceptions + } + if (NotReplicatedYetException.class.getName().equals(e.getClassName())) { + if (retries == 0) { + throw e; + } else { + --retries; + LOG.info("Exception while adding a block", e); + long elapsed = Time.monotonicNow() - localstart; + if (elapsed > 5000) { + LOG.info("Waiting for replication for " + (elapsed / 1000) + + " seconds"); + } + try { + LOG.warn("NotReplicatedYetException sleeping " + src + + " retries left " + retries); + Thread.sleep(sleeptime); + sleeptime *= 2; + } catch (InterruptedException ie) { + LOG.warn("Caught exception", ie); + } + } + } else { + throw e; + } + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index d3a054affeea0..c145a2a1fc8e7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -25,23 +25,34 @@ import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; -import org.apache.hadoop.hdfs.client.impl.DfsClientConf; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; +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.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.Time; import org.apache.htrace.Sampler; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; @@ -59,23 +70,11 @@ static class MultipleBlockingQueue { private final List> queues; MultipleBlockingQueue(int numQueue, int queueSize) { - queues = new ArrayList<>(numQueue); + List> list = new ArrayList<>(numQueue); for (int i = 0; i < numQueue; i++) { - queues.add(new LinkedBlockingQueue(queueSize)); + list.add(new LinkedBlockingQueue(queueSize)); } - } - - boolean isEmpty() { - for(int i = 0; i < queues.size(); i++) { - if (!queues.get(i).isEmpty()) { - return false; - } - } - return true; - } - - int numQueues() { - return queues.size(); + queues = Collections.synchronizedList(list); } void offer(int i, T object) { @@ -92,6 +91,14 @@ T take(int i) throws InterruptedIOException { } } + T takeWithTimeout(int i) throws InterruptedIOException { + try { + return queues.get(i).poll(100, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + throw DFSUtil.toInterruptedIOException("take interrupted, i=" + i, e); + } + } + T poll(int i) { return queues.get(i).poll(); } @@ -99,23 +106,44 @@ T poll(int i) { T peek(int i) { return queues.get(i).peek(); } + + void clear() { + for (BlockingQueue q : queues) { + q.clear(); + } + } } /** Coordinate the communication between the streamers. */ - class Coordinator { + static class Coordinator { + /** + * The next internal block to write to for each streamers. The + * DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to + * get a new block group. The block group is split to internal blocks, which + * are then distributed into the queue for streamers to retrieve. + */ private final MultipleBlockingQueue followingBlocks; + /** + * Used to sync among all the streamers before allocating a new block. The + * DFSStripedOutputStream uses this to make sure every streamer has finished + * writing the previous block. + */ private final MultipleBlockingQueue endBlocks; + /** + * The following data structures are used for syncing while handling errors + */ private final MultipleBlockingQueue newBlocks; - private final MultipleBlockingQueue updateBlocks; + private final Map updateStreamerMap; + private final MultipleBlockingQueue streamerUpdateResult; - Coordinator(final DfsClientConf conf, final int numDataBlocks, - final int numAllBlocks) { + Coordinator(final int numAllBlocks) { followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); - endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1); - + endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); - updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); + updateStreamerMap = Collections.synchronizedMap( + new HashMap(numAllBlocks)); + streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1); } MultipleBlockingQueue getFollowingBlocks() { @@ -126,68 +154,28 @@ MultipleBlockingQueue getNewBlocks() { return newBlocks; } - MultipleBlockingQueue getUpdateBlocks() { - return updateBlocks; - } - - StripedDataStreamer getStripedDataStreamer(int i) { - return DFSStripedOutputStream.this.getStripedDataStreamer(i); - } - void offerEndBlock(int i, ExtendedBlock block) { endBlocks.offer(i, block); } - ExtendedBlock takeEndBlock(int i) throws InterruptedIOException { - return endBlocks.take(i); + void offerStreamerUpdateResult(int i, boolean success) { + streamerUpdateResult.offer(i, success); } - boolean hasAllEndBlocks() { - for(int i = 0; i < endBlocks.numQueues(); i++) { - if (endBlocks.peek(i) == null) { - return false; - } - } - return true; + boolean takeStreamerUpdateResult(int i) throws InterruptedIOException { + return streamerUpdateResult.take(i); } - void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) { - ExtendedBlock b = endBlocks.peek(i); - if (b == null) { - // streamer just has failed, put end block and continue - b = block; - offerEndBlock(i, b); - } - b.setNumBytes(newBytes); + void updateStreamer(StripedDataStreamer streamer, + boolean success) { + assert !updateStreamerMap.containsKey(streamer); + updateStreamerMap.put(streamer, success); } - /** @return a block representing the entire block group. */ - ExtendedBlock getBlockGroup() { - final StripedDataStreamer s0 = getStripedDataStreamer(0); - final ExtendedBlock b0 = s0.getBlock(); - if (b0 == null) { - return null; - } - - final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0; - - final ExtendedBlock block = new ExtendedBlock(b0); - long numBytes = atBlockGroupBoundary? b0.getNumBytes(): s0.getBytesCurBlock(); - for (int i = 1; i < numAllBlocks; i++) { - final StripedDataStreamer si = getStripedDataStreamer(i); - final ExtendedBlock bi = si.getBlock(); - if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) { - block.setGenerationStamp(bi.getGenerationStamp()); - } - if (i < numDataBlocks) { - numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock(); - } - } - block.setNumBytes(numBytes); - if (LOG.isDebugEnabled()) { - LOG.debug("getBlockGroup: " + block + ", numBytes=" + block.getNumBytes()); - } - return block; + void clearFailureStates() { + newBlocks.clear(); + updateStreamerMap.clear(); + streamerUpdateResult.clear(); } } @@ -263,18 +251,16 @@ private void flipDataBuffers() { private final int cellSize; private final int numAllBlocks; private final int numDataBlocks; - - @Override - ExtendedBlock getBlock() { - return coordinator.getBlockGroup(); - } + private ExtendedBlock currentBlockGroup; + private final String[] favoredNodes; + private final List failedStreamers; /** Construct a new output stream for creating a file. */ DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, EnumSet flag, Progressable progress, DataChecksum checksum, String[] favoredNodes) throws IOException { - super(dfsClient, src, stat, flag, progress, checksum, favoredNodes); + super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false); if (LOG.isDebugEnabled()) { LOG.debug("Creating DFSStripedOutputStream for " + src); } @@ -284,12 +270,13 @@ ExtendedBlock getBlock() { cellSize = ecPolicy.getCellSize(); numDataBlocks = ecPolicy.getNumDataUnits(); numAllBlocks = numDataBlocks + numParityBlocks; + this.favoredNodes = favoredNodes; + failedStreamers = new ArrayList<>(); encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(), numDataBlocks, numParityBlocks); - coordinator = new Coordinator(dfsClient.getConf(), - numDataBlocks, numAllBlocks); + coordinator = new Coordinator(numAllBlocks); try { cellBuffers = new CellBuffers(numParityBlocks); } catch (InterruptedException ie) { @@ -297,14 +284,13 @@ ExtendedBlock getBlock() { "Failed to create cell buffers", ie); } - List s = new ArrayList<>(numAllBlocks); + streamers = new ArrayList<>(numAllBlocks); for (short i = 0; i < numAllBlocks; i++) { StripedDataStreamer streamer = new StripedDataStreamer(stat, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager, favoredNodes, i, coordinator); - s.add(streamer); + streamers.add(streamer); } - streamers = Collections.unmodifiableList(s); currentPackets = new DFSPacket[streamers.size()]; setCurrentStreamer(0); } @@ -318,17 +304,19 @@ int getCurrentIndex() { } private synchronized StripedDataStreamer getCurrentStreamer() { - return (StripedDataStreamer)streamer; + return (StripedDataStreamer) streamer; } private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) { // backup currentPacket for current streamer - int oldIdx = streamers.indexOf(streamer); - if (oldIdx >= 0) { - currentPackets[oldIdx] = currentPacket; + if (streamer != null) { + int oldIdx = streamers.indexOf(getCurrentStreamer()); + if (oldIdx >= 0) { + currentPackets[oldIdx] = currentPacket; + } } - streamer = streamers.get(newIdx); + streamer = getStripedDataStreamer(newIdx); currentPacket = currentPackets[newIdx]; adjustChunkBoundary(); @@ -350,40 +338,127 @@ private static void encode(RawErasureEncoder encoder, int numData, encoder.encode(dataBuffers, parityBuffers); } - - private void checkStreamers(boolean setExternalError) throws IOException { - int count = 0; + /** + * check all the existing StripedDataStreamer and find newly failed streamers. + * @return The newly failed streamers. + * @throws IOException if less than {@link #numDataBlocks} streamers are still + * healthy. + */ + private Set checkStreamers() throws IOException { + Set newFailed = new HashSet<>(); for(StripedDataStreamer s : streamers) { - if (!s.isFailed()) { - if (setExternalError && s.getBlock() != null) { - s.getErrorState().initExternalError(); - } - count++; + if (!s.isHealthy() && !failedStreamers.contains(s)) { + newFailed.add(s); } } + + final int failCount = failedStreamers.size() + newFailed.size(); if (LOG.isDebugEnabled()) { LOG.debug("checkStreamers: " + streamers); - LOG.debug("count=" + count); + LOG.debug("healthy streamer count=" + (numAllBlocks - failCount)); + LOG.debug("original failed streamers: " + failedStreamers); + LOG.debug("newly failed streamers: " + newFailed); } - if (count < numDataBlocks) { - throw new IOException("Failed: the number of remaining blocks = " - + count + " < the number of data blocks = " + numDataBlocks); + if (failCount > (numAllBlocks - numDataBlocks)) { + throw new IOException("Failed: the number of failed blocks = " + + failCount + " > the number of data blocks = " + + (numAllBlocks - numDataBlocks)); } + return newFailed; } private void handleStreamerFailure(String err, Exception e) throws IOException { - handleStreamerFailure(err, e, true); - } - - private void handleStreamerFailure(String err, Exception e, - boolean setExternalError) throws IOException { LOG.warn("Failed: " + err + ", " + this, e); - getCurrentStreamer().setFailed(true); - checkStreamers(setExternalError); + getCurrentStreamer().getErrorState().setInternalError(); + getCurrentStreamer().close(true); + checkStreamers(); currentPacket = null; } + private void replaceFailedStreamers() { + assert streamers.size() == numAllBlocks; + for (short i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer oldStreamer = getStripedDataStreamer(i); + if (!oldStreamer.isHealthy()) { + StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat, + dfsClient, src, oldStreamer.progress, + oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager, + favoredNodes, i, coordinator); + streamers.set(i, streamer); + currentPackets[i] = null; + if (i == 0) { + this.streamer = streamer; + } + streamer.start(); + } + } + } + + private void waitEndBlocks(int i) throws IOException { + while (getStripedDataStreamer(i).isHealthy()) { + final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i); + if (b != null) { + StripedBlockUtil.checkBlocks(currentBlockGroup, i, b); + return; + } + } + } + + private void allocateNewBlock() throws IOException { + if (currentBlockGroup != null) { + for (int i = 0; i < numAllBlocks; i++) { + // sync all the healthy streamers before writing to the new block + waitEndBlocks(i); + } + } + failedStreamers.clear(); + // replace failed streamers + replaceFailedStreamers(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Allocating new block group. The previous block group: " + + currentBlockGroup); + } + + // TODO collect excludedNodes from all the data streamers + final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup, + fileId, favoredNodes); + assert lb.isStriped(); + if (lb.getLocations().length < numDataBlocks) { + throw new IOException("Failed to get " + numDataBlocks + + " nodes from namenode: blockGroupSize= " + numAllBlocks + + ", blocks.length= " + lb.getLocations().length); + } + // assign the new block to the current block group + currentBlockGroup = lb.getBlock(); + + final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) lb, cellSize, numDataBlocks, + numAllBlocks - numDataBlocks); + for (int i = 0; i < blocks.length; i++) { + StripedDataStreamer si = getStripedDataStreamer(i); + if (si.isHealthy()) { // skipping failed data streamer + if (blocks[i] == null) { + // Set exception and close streamer as there is no block locations + // found for the parity block. + LOG.warn("Failed to get block location for parity block, index=" + i); + si.getLastException().set( + new IOException("Failed to get following block, i=" + i)); + si.getErrorState().setInternalError(); + si.close(true); + } else { + coordinator.getFollowingBlocks().offer(i, blocks[i]); + } + } + } + } + + private boolean shouldEndBlockGroup() { + return currentBlockGroup != null && + currentBlockGroup.getNumBytes() == blockSize * numDataBlocks; + } + @Override protected synchronized void writeChunk(byte[] bytes, int offset, int len, byte[] checksum, int ckoff, int cklen) throws IOException { @@ -392,8 +467,13 @@ protected synchronized void writeChunk(byte[] bytes, int offset, int len, final int pos = cellBuffers.addTo(index, bytes, offset, len); final boolean cellFull = pos == cellSize; - final long oldBytes = current.getBytesCurBlock(); - if (!current.isFailed()) { + if (currentBlockGroup == null || shouldEndBlockGroup()) { + // the incoming data should belong to a new block. Allocate a new block. + allocateNewBlock(); + } + + currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len); + if (current.isHealthy()) { try { super.writeChunk(bytes, offset, len, checksum, ckoff, cklen); } catch(Exception e) { @@ -401,12 +481,6 @@ protected synchronized void writeChunk(byte[] bytes, int offset, int len, } } - if (current.isFailed()) { - final long newBytes = oldBytes + len; - coordinator.setBytesEndBlock(index, newBytes, current.getBlock()); - current.setBytesCurBlock(newBytes); - } - // Two extra steps are needed when a striping cell is full: // 1. Forward the current index pointer // 2. Generate parity packets if a full stripe of data cells are present @@ -419,11 +493,209 @@ protected synchronized void writeChunk(byte[] bytes, int offset, int len, cellBuffers.flipDataBuffers(); writeParityCells(); next = 0; + // check failure state for all the streamers. Bump GS if necessary + checkStreamerFailures(); + + // if this is the end of the block group, end each internal block + if (shouldEndBlockGroup()) { + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer s = setCurrentStreamer(i); + if (s.isHealthy()) { + try { + endBlock(); + } catch (IOException ignored) {} + } + } + } } setCurrentStreamer(next); } } + @Override + void enqueueCurrentPacketFull() throws IOException { + LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={}," + + " appendChunk={}, {}", currentPacket, src, getStreamer() + .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(), + getStreamer()); + enqueueCurrentPacket(); + adjustChunkBoundary(); + // no need to end block here + } + + private Set markExternalErrorOnStreamers() { + Set healthySet = new HashSet<>(); + for (StripedDataStreamer streamer : streamers) { + if (streamer.isHealthy() && + streamer.getStage() == BlockConstructionStage.DATA_STREAMING) { + streamer.setExternalError(); + healthySet.add(streamer); + } + } + return healthySet; + } + + /** + * Check and handle data streamer failures. This is called only when we have + * written a full stripe (i.e., enqueue all packets for a full stripe), or + * when we're closing the outputstream. + */ + private void checkStreamerFailures() throws IOException { + Set newFailed = checkStreamers(); + if (newFailed.size() > 0) { + // for healthy streamers, wait till all of them have fetched the new block + // and flushed out all the enqueued packets. + flushAllInternals(); + } + // get all the current failed streamers after the flush + newFailed = checkStreamers(); + while (newFailed.size() > 0) { + failedStreamers.addAll(newFailed); + coordinator.clearFailureStates(); + + // mark all the healthy streamers as external error + Set healthySet = markExternalErrorOnStreamers(); + + // we have newly failed streamers, update block for pipeline + final ExtendedBlock newBG = updateBlockForPipeline(healthySet); + + // wait till all the healthy streamers to + // 1) get the updated block info + // 2) create new block outputstream + newFailed = waitCreatingNewStreams(healthySet); + if (newFailed.size() + failedStreamers.size() > + numAllBlocks - numDataBlocks) { + throw new IOException( + "Data streamers failed while creating new block streams: " + + newFailed + ". There are not enough healthy streamers."); + } + for (StripedDataStreamer failedStreamer : newFailed) { + assert !failedStreamer.isHealthy(); + } + + // TODO we can also succeed if all the failed streamers have not taken + // the updated block + if (newFailed.size() == 0) { + // reset external error state of all the streamers + for (StripedDataStreamer streamer : healthySet) { + assert streamer.isHealthy(); + streamer.getErrorState().reset(); + } + updatePipeline(newBG); + } + for (int i = 0; i < numAllBlocks; i++) { + coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0); + } + } + } + + private int checkStreamerUpdates(Set failed, + Set streamers) { + for (StripedDataStreamer streamer : streamers) { + if (!coordinator.updateStreamerMap.containsKey(streamer)) { + if (!streamer.isHealthy() && + coordinator.getNewBlocks().peek(streamer.getIndex()) != null) { + // this streamer had internal error before getting updated block + failed.add(streamer); + } + } + } + return coordinator.updateStreamerMap.size() + failed.size(); + } + + private Set waitCreatingNewStreams( + Set healthyStreamers) throws IOException { + Set failed = new HashSet<>(); + final int expectedNum = healthyStreamers.size(); + final long socketTimeout = dfsClient.getConf().getSocketTimeout(); + // the total wait time should be less than the socket timeout, otherwise + // a slow streamer may cause other streamers to timeout. here we wait for + // half of the socket timeout + long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE; + final long waitInterval = 1000; + synchronized (coordinator) { + while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum + && remaingTime > 0) { + try { + long start = Time.monotonicNow(); + coordinator.wait(waitInterval); + remaingTime -= Time.monotonicNow() - start; + } catch (InterruptedException e) { + throw DFSUtil.toInterruptedIOException("Interrupted when waiting" + + " for results of updating striped streamers", e); + } + } + } + synchronized (coordinator) { + for (StripedDataStreamer streamer : healthyStreamers) { + if (!coordinator.updateStreamerMap.containsKey(streamer)) { + // close the streamer if it is too slow to create new connection + streamer.setStreamerAsClosed(); + failed.add(streamer); + } + } + } + for (Map.Entry entry : + coordinator.updateStreamerMap.entrySet()) { + if (!entry.getValue()) { + failed.add(entry.getKey()); + } + } + for (StripedDataStreamer failedStreamer : failed) { + healthyStreamers.remove(failedStreamer); + } + return failed; + } + + /** + * Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block + * to healthy streamers. + * @param healthyStreamers The healthy data streamers. These streamers join + * the failure handling. + */ + private ExtendedBlock updateBlockForPipeline( + Set healthyStreamers) throws IOException { + final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline( + currentBlockGroup, dfsClient.clientName); + final long newGS = updated.getBlock().getGenerationStamp(); + ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup); + newBlock.setGenerationStamp(newGS); + final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) updated, cellSize, numDataBlocks, + numAllBlocks - numDataBlocks); + + for (int i = 0; i < numAllBlocks; i++) { + StripedDataStreamer si = getStripedDataStreamer(i); + if (healthyStreamers.contains(si)) { + final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock), + null, null, null, -1, updated.isCorrupt(), null); + lb.setBlockToken(updatedBlks[i].getBlockToken()); + coordinator.getNewBlocks().offer(i, lb); + } + } + return newBlock; + } + + private void updatePipeline(ExtendedBlock newBG) throws IOException { + final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks]; + final String[] newStorageIDs = new String[numAllBlocks]; + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer streamer = getStripedDataStreamer(i); + final DatanodeInfo[] nodes = streamer.getNodes(); + final String[] storageIDs = streamer.getStorageIDs(); + if (streamer.isHealthy() && nodes != null && storageIDs != null) { + newNodes[i] = nodes[0]; + newStorageIDs[i] = storageIDs[0]; + } else { + newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID); + newStorageIDs[i] = ""; + } + } + dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup, + newBG, newNodes, newStorageIDs); + currentBlockGroup = newBG; + } + private int stripeDataSize() { return numDataBlocks * cellSize; } @@ -500,28 +772,16 @@ protected void closeThreads(boolean force) throws IOException { } } - /** - * Simply add bytesCurBlock together. Note that this result is not accurately - * the size of the block group. - */ - private long getCurrentSumBytes() { - long sum = 0; - for (int i = 0; i < numDataBlocks; i++) { - sum += streamers.get(i).getBytesCurBlock(); - } - return sum; - } - private boolean generateParityCellsForLastStripe() { - final long currentBlockGroupBytes = getCurrentSumBytes(); - if (currentBlockGroupBytes % stripeDataSize() == 0) { + final long currentBlockGroupBytes = currentBlockGroup == null ? + 0 : currentBlockGroup.getNumBytes(); + final long lastStripeSize = currentBlockGroupBytes % stripeDataSize(); + if (lastStripeSize == 0) { return false; } - final int firstCellSize = - (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize); - final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize? - firstCellSize : cellSize; + final long parityCellSize = lastStripeSize < cellSize? + lastStripeSize : cellSize; final ByteBuffer[] buffers = cellBuffers.getBuffers(); for (int i = 0; i < numAllBlocks; i++) { @@ -550,13 +810,13 @@ void writeParityCells() throws IOException { cellBuffers.clear(); } - void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf - ) throws IOException { + void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf) + throws IOException { final StripedDataStreamer current = setCurrentStreamer(index); final int len = buffer.limit(); final long oldBytes = current.getBytesCurBlock(); - if (!current.isFailed()) { + if (current.isHealthy()) { try { DataChecksum sum = getDataChecksum(); sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0); @@ -570,18 +830,13 @@ void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e); } } - - if (current.isFailed()) { - final long newBytes = oldBytes + len; - current.setBytesCurBlock(newBytes); - } } @Override void setClosed() { super.setClosed(); for (int i = 0; i < numAllBlocks; i++) { - streamers.get(i).release(); + getStripedDataStreamer(i).release(); } cellBuffers.release(); } @@ -607,37 +862,40 @@ protected synchronized void closeImpl() throws IOException { try { // flush from all upper layers - try { - flushBuffer(); - } catch(Exception e) { - handleStreamerFailure("flushBuffer " + getCurrentStreamer(), e); - } + flushBuffer(); // if the last stripe is incomplete, generate and write parity cells if (generateParityCellsForLastStripe()) { writeParityCells(); } enqueueAllCurrentPackets(); + // flush all the data packets + flushAllInternals(); + // check failures + checkStreamerFailures(); + for (int i = 0; i < numAllBlocks; i++) { final StripedDataStreamer s = setCurrentStreamer(i); - if (!s.isFailed()) { + if (s.isHealthy()) { try { if (s.getBytesCurBlock() > 0) { setCurrentPacketToEmpty(); } - // flush all data to Datanode + // flush the last "close" packet to Datanode flushInternal(); } catch(Exception e) { - handleStreamerFailure("flushInternal " + s, e, false); + // TODO for both close and endBlock, we currently do not handle + // failures when sending the last packet. We actually do not need to + // bump GS for this kind of failure. Thus counting the total number + // of failures may be good enough. } } } closeThreads(false); - final ExtendedBlock lastBlock = coordinator.getBlockGroup(); TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER); try { - completeFile(lastBlock); + completeFile(currentBlockGroup); } finally { scope.close(); } @@ -652,14 +910,45 @@ private void enqueueAllCurrentPackets() throws IOException { int idx = streamers.indexOf(getCurrentStreamer()); for(int i = 0; i < streamers.size(); i++) { final StripedDataStreamer si = setCurrentStreamer(i); - if (!si.isFailed() && currentPacket != null) { + if (si.isHealthy() && currentPacket != null) { try { enqueueCurrentPacket(); } catch (IOException e) { - handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e, false); + handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e); } } } setCurrentStreamer(idx); } + + void flushAllInternals() throws IOException { + int current = getCurrentIndex(); + + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer s = setCurrentStreamer(i); + if (s.isHealthy()) { + try { + // flush all data to Datanode + flushInternal(); + } catch(Exception e) { + handleStreamerFailure("flushInternal " + s, e); + } + } + } + setCurrentStreamer(current); + } + + static void sleep(long ms, String op) throws InterruptedIOException { + try { + Thread.sleep(ms); + } catch(InterruptedException ie) { + throw DFSUtil.toInterruptedIOException( + "Sleep interrupted during " + op, ie); + } + } + + @Override + ExtendedBlock getBlock() { + return currentBlockGroup; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index c478f1c3f7060..a6eb01f989cd9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -22,7 +22,6 @@ import java.io.BufferedOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.InterruptedIOException; @@ -46,16 +45,12 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite; import org.apache.hadoop.hdfs.client.impl.DfsClientConf; -import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; 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.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; -import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; -import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; @@ -69,13 +64,10 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; -import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException; import org.apache.hadoop.hdfs.util.ByteArrayManager; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MultipleIOException; -import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.DataChecksum; @@ -204,9 +196,12 @@ synchronized void throwException4Close() throws IOException { } } + enum ErrorType { + NONE, INTERNAL, EXTERNAL + } + static class ErrorState { - private boolean error = false; - private boolean externalError = false; + ErrorType error = ErrorType.NONE; private int badNodeIndex = -1; private int restartingNodeIndex = -1; private long restartingNodeDeadline = 0; @@ -216,35 +211,47 @@ static class ErrorState { this.datanodeRestartTimeout = datanodeRestartTimeout; } + synchronized void resetInternalError() { + if (hasInternalError()) { + error = ErrorType.NONE; + } + badNodeIndex = -1; + restartingNodeIndex = -1; + restartingNodeDeadline = 0; + } + synchronized void reset() { - error = false; - externalError = false; + error = ErrorType.NONE; badNodeIndex = -1; restartingNodeIndex = -1; restartingNodeDeadline = 0; } - synchronized boolean hasError() { - return error; + synchronized boolean hasInternalError() { + return error == ErrorType.INTERNAL; } - synchronized boolean hasExternalErrorOnly() { - return error && externalError && !isNodeMarked(); + synchronized boolean hasExternalError() { + return error == ErrorType.EXTERNAL; } - synchronized boolean hasDatanodeError() { - return error && (isNodeMarked() || externalError); + synchronized boolean hasError() { + return error != ErrorType.NONE; } - synchronized void setError(boolean err) { - this.error = err; + synchronized boolean hasDatanodeError() { + return error == ErrorType.INTERNAL && isNodeMarked(); } - synchronized void initExternalError() { - setError(true); - this.externalError = true; + synchronized void setInternalError() { + this.error = ErrorType.INTERNAL; } + synchronized void setExternalError() { + if (!hasInternalError()) { + this.error = ErrorType.EXTERNAL; + } + } synchronized void setBadNodeIndex(int index) { this.badNodeIndex = index; @@ -306,14 +313,14 @@ synchronized void adjustState4RestartingNode() { } if (!isRestartingNode()) { - error = false; + error = ErrorType.NONE; } badNodeIndex = -1; } synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { if (restartingNodeIndex >= 0) { - if (!error) { + if (error == ErrorType.NONE) { throw new IllegalStateException("error=false while checking" + " restarting node deadline"); } @@ -345,7 +352,7 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { private volatile boolean streamerClosed = false; protected ExtendedBlock block; // its length is number of bytes acked - private Token accessToken; + protected Token accessToken; private DataOutputStream blockStream; private DataInputStream blockReplyStream; private ResponseProcessor response = null; @@ -355,7 +362,7 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { private final ErrorState errorState; private BlockConstructionStage stage; // block construction stage - private long bytesSent = 0; // number of bytes that've been sent + protected long bytesSent = 0; // number of bytes that've been sent private final boolean isLazyPersistFile; /** Nodes have been used in the pipeline before and have failed. */ @@ -378,13 +385,13 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { protected final DFSClient dfsClient; protected final String src; /** Only for DataTransferProtocol.writeBlock(..) */ - private final DataChecksum checksum4WriteBlock; - private final Progressable progress; + final DataChecksum checksum4WriteBlock; + final Progressable progress; protected final HdfsFileStatus stat; // appending to existing partial block private volatile boolean appendChunk = false; // both dataQueue and ackQueue are protected by dataQueue lock - private final LinkedList dataQueue = new LinkedList<>(); + protected final LinkedList dataQueue = new LinkedList<>(); private final LinkedList ackQueue = new LinkedList<>(); private final AtomicReference cachingStrategy; private final ByteArrayManager byteArrayManager; @@ -401,7 +408,7 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10; private int lastCongestionBackoffTime; - private final LoadingCache excludedNodes; + protected final LoadingCache excludedNodes; private final String[] favoredNodes; private DataStreamer(HdfsFileStatus stat, ExtendedBlock block, @@ -473,6 +480,10 @@ void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{ } } + void setAccessToken(Token t) { + this.accessToken = t; + } + private void setPipeline(LocatedBlock lb) { setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs()); } @@ -533,7 +544,7 @@ public void run() { DFSPacket one; try { // process datanode IO errors if any - boolean doSleep = processDatanodeError(); + boolean doSleep = processDatanodeOrExternalError(); final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; synchronized (dataQueue) { @@ -696,7 +707,7 @@ public void run() { } lastException.set(e); assert !(e instanceof NullPointerException); - errorState.setError(true); + errorState.setInternalError(); if (!errorState.isNodeMarked()) { // Not a datanode issue streamerClosed = true; @@ -837,6 +848,9 @@ void close(boolean force) { } } + void setStreamerAsClosed() { + streamerClosed = true; + } private void checkClosed() throws IOException { if (streamerClosed) { @@ -857,7 +871,7 @@ private void closeResponder() { } } - private void closeStream() { + void closeStream() { final MultipleIOException.Builder b = new MultipleIOException.Builder(); if (blockStream != null) { @@ -1037,7 +1051,7 @@ public void run() { } catch (Exception e) { if (!responderClosed) { lastException.set(e); - errorState.setError(true); + errorState.setInternalError(); errorState.markFirstNodeIfNotMarked(); synchronized (dataQueue) { dataQueue.notifyAll(); @@ -1059,18 +1073,18 @@ void close() { } } + private boolean shouldHandleExternalError(){ + return errorState.hasExternalError() && blockStream != null; + } + /** * If this stream has encountered any errors, shutdown threads * and mark the stream as closed. * * @return true if it should sleep for a while after returning. */ - private boolean processDatanodeError() throws IOException { - if (!errorState.hasDatanodeError()) { - return false; - } - if (errorState.hasExternalErrorOnly() && block == null) { - // block is not yet initialized, handle external error later. + private boolean processDatanodeOrExternalError() throws IOException { + if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) { return false; } if (response != null) { @@ -1103,7 +1117,8 @@ private boolean processDatanodeError() throws IOException { return false; } } - boolean doSleep = setupPipelineForAppendOrRecovery(); + + setupPipelineForAppendOrRecovery(); if (!streamerClosed && dfsClient.clientRunning) { if (stage == BlockConstructionStage.PIPELINE_CLOSE) { @@ -1135,7 +1150,7 @@ private boolean processDatanodeError() throws IOException { } } - return doSleep; + return false; } void setHflush() { @@ -1266,7 +1281,7 @@ private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets, * This happens when a file is appended or data streaming fails * It keeps on trying until a pipeline is setup */ - private boolean setupPipelineForAppendOrRecovery() throws IOException { + private void setupPipelineForAppendOrRecovery() throws IOException { // check number of datanodes if (nodes == null || nodes.length == 0) { String msg = "Could not get block locations. " + "Source file \"" @@ -1274,19 +1289,23 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException { LOG.warn(msg); lastException.set(new IOException(msg)); streamerClosed = true; - return false; + return; } + setupPipelineInternal(nodes, storageTypes); + } + protected void setupPipelineInternal(DatanodeInfo[] datanodes, + StorageType[] nodeStorageTypes) throws IOException { boolean success = false; long newGS = 0L; while (!success && !streamerClosed && dfsClient.clientRunning) { if (!handleRestartingDatanode()) { - return false; + return; } - final boolean isRecovery = errorState.hasError(); + final boolean isRecovery = errorState.hasInternalError(); if (!handleBadDatanode()) { - return false; + return; } handleDatanodeReplacement(); @@ -1307,7 +1326,6 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException { if (success) { block = updatePipeline(newGS); } - return false; // do not sleep, continue processing } /** @@ -1315,7 +1333,7 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException { * This process is repeated until the deadline or the node starts back up. * @return true if it should continue. */ - private boolean handleRestartingDatanode() { + boolean handleRestartingDatanode() { if (errorState.isRestartingNode()) { // 4 seconds or the configured deadline period, whichever is shorter. // This is the retry interval and recovery will be retried in this @@ -1338,7 +1356,7 @@ private boolean handleRestartingDatanode() { * Remove bad node from list of nodes if badNodeIndex was set. * @return true if it should continue. */ - private boolean handleBadDatanode() { + boolean handleBadDatanode() { final int badNodeIndex = errorState.getBadNodeIndex(); if (badNodeIndex >= 0) { if (nodes.length <= 1) { @@ -1388,7 +1406,7 @@ private void handleDatanodeReplacement() throws IOException { } } - private void failPacket4Testing() { + void failPacket4Testing() { if (failPacket) { // for testing failPacket = false; try { @@ -1400,13 +1418,8 @@ private void failPacket4Testing() { } } - LocatedBlock updateBlockForPipeline() throws IOException { - return callUpdateBlockForPipeline(block); - } - - LocatedBlock callUpdateBlockForPipeline(ExtendedBlock newBlock) throws IOException { - return dfsClient.namenode.updateBlockForPipeline( - newBlock, dfsClient.clientName); + private LocatedBlock updateBlockForPipeline() throws IOException { + return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName); } static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) { @@ -1417,18 +1430,12 @@ static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) { /** update pipeline at the namenode */ ExtendedBlock updatePipeline(long newGS) throws IOException { final ExtendedBlock newBlock = newBlock(block, newGS); - return callUpdatePipeline(block, newBlock, nodes, storageIDs); - } - - ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock, - DatanodeInfo[] newNodes, String[] newStorageIDs) - throws IOException { - dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock, newBlock, - newNodes, newStorageIDs); + dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, + nodes, storageIDs); return newBlock; } - int getNumBlockWriteRetry() { + private int getNumBlockWriteRetry() { return dfsClient.getConf().getNumBlockWriteRetry(); } @@ -1438,7 +1445,7 @@ int getNumBlockWriteRetry() { * Must get block ID and the IDs of the destinations from the namenode. * Returns the list of target datanodes. */ - private LocatedBlock nextBlockOutputStream() throws IOException { + protected LocatedBlock nextBlockOutputStream() throws IOException { LocatedBlock lb = null; DatanodeInfo[] nodes = null; StorageType[] storageTypes = null; @@ -1446,9 +1453,8 @@ private LocatedBlock nextBlockOutputStream() throws IOException { boolean success = false; ExtendedBlock oldBlock = block; do { - errorState.reset(); + errorState.resetInternalError(); lastException.clear(); - success = false; DatanodeInfo[] excluded = excludedNodes.getAllPresent(excludedNodes.asMap().keySet()) @@ -1488,7 +1494,7 @@ private LocatedBlock nextBlockOutputStream() throws IOException { // connects to the first datanode in the pipeline // Returns true if success, otherwise return failure. // - private boolean createBlockOutputStream(DatanodeInfo[] nodes, + boolean createBlockOutputStream(DatanodeInfo[] nodes, StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) { if (nodes.length == 0) { LOG.info("nodes are empty for write pipeline of " + block); @@ -1567,7 +1573,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, assert null == blockStream : "Previous blockStream unclosed"; blockStream = out; result = true; // success - errorState.reset(); + errorState.resetInternalError(); } catch (IOException ie) { if (!errorState.isRestartingNode()) { LOG.info("Exception in createBlockOutputStream " + this, ie); @@ -1603,7 +1609,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, if (checkRestart && shouldWaitForRestart(i)) { errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]); } - errorState.setError(true); + errorState.setInternalError(); lastException.set(ie); result = false; // error } finally { @@ -1645,58 +1651,10 @@ private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) { } } - LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) + private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) throws IOException { - final DfsClientConf conf = dfsClient.getConf(); - int retries = conf.getNumBlockWriteLocateFollowingRetry(); - long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs(); - while (true) { - long localstart = Time.monotonicNow(); - while (true) { - try { - return dfsClient.namenode.addBlock(src, dfsClient.clientName, - block, excludedNodes, stat.getFileId(), favoredNodes); - } catch (RemoteException e) { - IOException ue = - e.unwrapRemoteException(FileNotFoundException.class, - AccessControlException.class, - NSQuotaExceededException.class, - DSQuotaExceededException.class, - QuotaByStorageTypeExceededException.class, - UnresolvedPathException.class); - if (ue != e) { - throw ue; // no need to retry these exceptions - } - - - if (NotReplicatedYetException.class.getName(). - equals(e.getClassName())) { - if (retries == 0) { - throw e; - } else { - --retries; - LOG.info("Exception while adding a block", e); - long elapsed = Time.monotonicNow() - localstart; - if (elapsed > 5000) { - LOG.info("Waiting for replication for " - + (elapsed / 1000) + " seconds"); - } - try { - LOG.warn("NotReplicatedYetException sleeping " + src - + " retries left " + retries); - Thread.sleep(sleeptime); - sleeptime *= 2; - } catch (InterruptedException ie) { - LOG.warn("Caught exception", ie); - } - } - } else { - throw e; - } - - } - } - } + return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block, + stat.getFileId(), favoredNodes); } /** @@ -1755,6 +1713,10 @@ String[] getStorageIDs() { return storageIDs; } + BlockConstructionStage getStage() { + return stage; + } + /** * return the token of the block * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java index 2f83f7c641837..a313ecb18949f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -19,18 +19,15 @@ package org.apache.hadoop.hdfs; import java.io.IOException; -import java.io.InterruptedIOException; import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator; -import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -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.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.ByteArrayManager; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; @@ -46,66 +43,8 @@ * other streamers. */ public class StripedDataStreamer extends DataStreamer { - /** - * This class is designed for multiple threads to share a - * {@link MultipleBlockingQueue}. Initially, the queue is empty. The earliest - * thread calling poll populates entries to the queue and the other threads - * will wait for it. Once the entries are populated, all the threads can poll - * their entries. - * - * @param the queue entry type. - */ - static abstract class ConcurrentPoll { - final MultipleBlockingQueue queue; - - ConcurrentPoll(MultipleBlockingQueue queue) { - this.queue = queue; - } - - T poll(final int i) throws IOException { - for(;;) { - synchronized(queue) { - final T polled = queue.poll(i); - if (polled != null) { // already populated; return polled item. - return polled; - } - if (isReady2Populate()) { - try { - populate(); - return queue.poll(i); - } catch(IOException ioe) { - LOG.warn("Failed to populate, " + this, ioe); - throw ioe; - } - } - } - - // sleep and then retry. - sleep(100, "poll"); - } - } - - boolean isReady2Populate() { - return queue.isEmpty(); - } - - abstract void populate() throws IOException; - } - - private static void sleep(long ms, String op) throws InterruptedIOException { - try { - Thread.sleep(ms); - } catch(InterruptedException ie) { - throw DFSUtil.toInterruptedIOException( - "Sleep interrupted during " + op, ie); - } - } - private final Coordinator coordinator; private final int index; - private volatile boolean failed; - private final ECSchema schema; - private final int cellSize; StripedDataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, @@ -117,102 +56,59 @@ private static void sleep(long ms, String op) throws InterruptedIOException { byteArrayManage, favoredNodes); this.index = index; this.coordinator = coordinator; - this.schema = stat.getErasureCodingPolicy().getSchema(); - this.cellSize = stat.getErasureCodingPolicy().getCellSize(); } int getIndex() { return index; } - void setFailed(boolean failed) { - this.failed = failed; - } - - boolean isFailed() { - return failed; - } - - private boolean isParityStreamer() { - return index >= schema.getNumDataUnits(); + boolean isHealthy() { + return !streamerClosed() && !getErrorState().hasInternalError(); } @Override protected void endBlock() { - if (!isParityStreamer()) { - coordinator.offerEndBlock(index, block); - } + coordinator.offerEndBlock(index, block); super.endBlock(); } - @Override - int getNumBlockWriteRetry() { - return 0; + /** + * The upper level DFSStripedOutputStream will allocate the new block group. + * All the striped data streamer only needs to fetch from the queue, which + * should be already be ready. + */ + private LocatedBlock getFollowingBlock() throws IOException { + if (!this.isHealthy()) { + // No internal block for this streamer, maybe no enough healthy DN. + // Throw the exception which has been set by the StripedOutputStream. + this.getLastException().check(false); + } + return coordinator.getFollowingBlocks().poll(index); } @Override - LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes) - throws IOException { - return new ConcurrentPoll(coordinator.getFollowingBlocks()) { - @Override - boolean isReady2Populate() { - return super.isReady2Populate() - && (block == null || coordinator.hasAllEndBlocks()); - } - - @Override - void populate() throws IOException { - getLastException().check(false); - - if (block != null) { - // set numByte for the previous block group - long bytes = 0; - for (int i = 0; i < schema.getNumDataUnits(); i++) { - final ExtendedBlock b = coordinator.takeEndBlock(i); - StripedBlockUtil.checkBlocks(index, block, i, b); - bytes += b.getNumBytes(); - } - block.setNumBytes(bytes); - block.setBlockId(block.getBlockId() - index); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("locateFollowingBlock: index=" + index + ", block=" + block); - } - - final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock( - excludedNodes); - if (lb.getLocations().length < schema.getNumDataUnits()) { - throw new IOException( - "Failed to get datablocks number of nodes from namenode: blockGroupSize= " - + (schema.getNumDataUnits() + schema.getNumParityUnits()) - + ", blocks.length= " + lb.getLocations().length); - } - final LocatedBlock[] blocks = - StripedBlockUtil.parseStripedBlockGroup((LocatedStripedBlock) lb, - cellSize, schema.getNumDataUnits(), schema.getNumParityUnits()); - - for (int i = 0; i < blocks.length; i++) { - StripedDataStreamer si = coordinator.getStripedDataStreamer(i); - if (si.isFailed()) { - continue; // skipping failed data streamer - } - if (blocks[i] == null) { - // Set exception and close streamer as there is no block locations - // found for the parity block. - LOG.warn("Failed to get block location for parity block, index=" - + i); - si.getLastException().set( - new IOException("Failed to get following block, i=" + i)); - si.setFailed(true); - si.endBlock(); - si.close(true); - } else { - queue.offer(i, blocks[i]); - } - } - } - }.poll(index); + protected LocatedBlock nextBlockOutputStream() throws IOException { + boolean success; + LocatedBlock lb = getFollowingBlock(); + block = lb.getBlock(); + block.setNumBytes(0); + bytesSent = 0; + accessToken = lb.getBlockToken(); + + DatanodeInfo[] nodes = lb.getLocations(); + StorageType[] storageTypes = lb.getStorageTypes(); + + // Connect to the DataNode. If fail the internal error state will be set. + success = createBlockOutputStream(nodes, storageTypes, 0L, false); + + if (!success) { + block = null; + final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()]; + LOG.info("Excluding datanode " + badNode); + excludedNodes.put(badNode, badNode); + throw new IOException("Unable to create new block."); + } + return lb; } @VisibleForTesting @@ -221,119 +117,71 @@ LocatedBlock peekFollowingBlock() { } @Override - LocatedBlock updateBlockForPipeline() throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("updateBlockForPipeline(), " + this); - } - return new ConcurrentPoll(coordinator.getNewBlocks()) { - @Override - void populate() throws IOException { - final ExtendedBlock bg = coordinator.getBlockGroup(); - final LocatedBlock updated = callUpdateBlockForPipeline(bg); - final long newGS = updated.getBlock().getGenerationStamp(); - final LocatedBlock[] updatedBlks = StripedBlockUtil - .parseStripedBlockGroup((LocatedStripedBlock) updated, cellSize, - schema.getNumDataUnits(), schema.getNumParityUnits()); - for (int i = 0; i < schema.getNumDataUnits() - + schema.getNumParityUnits(); i++) { - StripedDataStreamer si = coordinator.getStripedDataStreamer(i); - if (si.isFailed()) { - continue; // skipping failed data streamer - } - final ExtendedBlock bi = si.getBlock(); - if (bi != null) { - final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS), - null, null, null, -1, updated.isCorrupt(), null); - lb.setBlockToken(updatedBlks[i].getBlockToken()); - queue.offer(i, lb); - } else { - final MultipleBlockingQueue followingBlocks - = coordinator.getFollowingBlocks(); - synchronized(followingBlocks) { - final LocatedBlock lb = followingBlocks.peek(i); - if (lb != null) { - lb.getBlock().setGenerationStamp(newGS); - si.getErrorState().reset(); - continue; - } - } - - //streamer i just have polled the block, sleep and retry. - sleep(100, "updateBlockForPipeline, " + this); - i--; - } - } + protected void setupPipelineInternal(DatanodeInfo[] nodes, + StorageType[] nodeStorageTypes) throws IOException { + boolean success = false; + while (!success && !streamerClosed() && dfsClient.clientRunning) { + if (!handleRestartingDatanode()) { + return; + } + if (!handleBadDatanode()) { + // for striped streamer if it is datanode error then close the stream + // and return. no need to replace datanode + return; } - }.poll(index); - } - - @Override - ExtendedBlock updatePipeline(final long newGS) throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("updatePipeline(newGS=" + newGS + "), " + this); - } - return new ConcurrentPoll(coordinator.getUpdateBlocks()) { - @Override - void populate() throws IOException { - final MultipleBlockingQueue followingBlocks - = coordinator.getFollowingBlocks(); - final ExtendedBlock bg = coordinator.getBlockGroup(); - final ExtendedBlock newBG = newBlock(bg, newGS); - final int n = schema.getNumDataUnits() + schema.getNumParityUnits(); - final DatanodeInfo[] newNodes = new DatanodeInfo[n]; - final String[] newStorageIDs = new String[n]; - for (int i = 0; i < n; i++) { - final StripedDataStreamer si = coordinator.getStripedDataStreamer(i); - DatanodeInfo[] nodes = si.getNodes(); - String[] storageIDs = si.getStorageIDs(); - if (nodes == null || storageIDs == null) { - synchronized(followingBlocks) { - final LocatedBlock lb = followingBlocks.peek(i); - if (lb != null) { - nodes = lb.getLocations(); - storageIDs = lb.getStorageIDs(); - } - } - } - if (nodes != null && storageIDs != null) { - newNodes[i] = nodes[0]; - newStorageIDs[i] = storageIDs[0]; - } else { - //streamer i just have polled the block, sleep and retry. - sleep(100, "updatePipeline, " + this); - i--; - } + // get a new generation stamp and an access token + final LocatedBlock lb = coordinator.getNewBlocks().take(index); + long newGS = lb.getBlock().getGenerationStamp(); + setAccessToken(lb.getBlockToken()); + + // set up the pipeline again with the remaining nodes. when a striped + // data streamer comes here, it must be in external error state. + assert getErrorState().hasExternalError(); + success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true); + + failPacket4Testing(); + getErrorState().checkRestartingNodeDeadline(nodes); + + // notify coordinator the result of createBlockOutputStream + synchronized (coordinator) { + if (!streamerClosed()) { + coordinator.updateStreamer(this, success); + coordinator.notify(); + } else { + success = false; } - final ExtendedBlock updated = callUpdatePipeline(bg, newBG, newNodes, - newStorageIDs); - - for (int i = 0; i < n; i++) { - final StripedDataStreamer si = coordinator.getStripedDataStreamer(i); - final ExtendedBlock bi = si.getBlock(); - if (bi != null) { - queue.offer(i, newBlock(bi, updated.getGenerationStamp())); - } else if (!si.isFailed()) { - synchronized(followingBlocks) { - final LocatedBlock lb = followingBlocks.peek(i); - if (lb != null) { - lb.getBlock().setGenerationStamp(newGS); - si.getErrorState().reset(); - continue; - } - } + } - //streamer i just have polled the block, sleep and retry. - sleep(100, "updatePipeline, " + this); - i--; - } + if (success) { + // wait for results of other streamers + success = coordinator.takeStreamerUpdateResult(index); + if (success) { + // if all succeeded, update its block using the new GS + block = newBlock(block, newGS); + } else { + // otherwise close the block stream and restart the recovery process + closeStream(); } + } else { + // if fail, close the stream. The internal error state and last + // exception have already been set in createBlockOutputStream + // TODO: wait for restarting DataNodes during RollingUpgrade + closeStream(); + setStreamerAsClosed(); } - }.poll(index); + } // while + } + + void setExternalError() { + getErrorState().setExternalError(); + synchronized (dataQueue) { + dataQueue.notifyAll(); + } } @Override public String toString() { - return "#" + index + ": " + (failed? "failed, ": "") + super.toString(); + return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java index 0e927797b3eca..1d4cff3340ea6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java @@ -68,16 +68,28 @@ assert getBlockUCState() != COMPLETE : /** Set expected locations */ public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets, boolean isStriped) { - int numLocations = targets == null ? 0 : targets.length; + if (targets == null) { + return; + } + int numLocations = 0; + for (DatanodeStorageInfo target : targets) { + if (target != null) { + numLocations++; + } + } + this.replicas = new ReplicaUnderConstruction[numLocations]; - for(int i = 0; i < numLocations; i++) { - // when creating a new striped block we simply sequentially assign block - // index to each storage - Block replicaBlock = isStriped ? - new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) : - block; - replicas[i] = new ReplicaUnderConstruction(replicaBlock, targets[i], - ReplicaState.RBW); + int offset = 0; + for(int i = 0; i < targets.length; i++) { + if (targets[i] != null) { + // when creating a new striped block we simply sequentially assign block + // index to each storage + Block replicaBlock = isStriped ? + new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) : + block; + replicas[offset++] = new ReplicaUnderConstruction(replicaBlock, + targets[i], ReplicaState.RBW); + } } } 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 b5b3b9727d391..61c6386f1439c 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 @@ -513,6 +513,10 @@ public DatanodeStorageInfo[] getDatanodeStorageInfos( } final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length]; for(int i = 0; i < datanodeID.length; i++) { + if (datanodeID[i].equals(DatanodeID.EMPTY_DATANODE_ID)) { + storages[i] = null; + continue; + } final DatanodeDescriptor dd = getDatanode(datanodeID[i]); storages[i] = dd.getStorageInfo(storageIDs[i]); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 5af35853d215e..d49d39bf19d3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -925,22 +925,21 @@ public String toString() { /** * Check if the information such as IDs and generation stamps in block-i - * match block-j, where block-i and block-j are in the same group. + * match the block group. */ - public static void checkBlocks(int j, ExtendedBlock blockj, + public static void checkBlocks(ExtendedBlock blockGroup, int i, ExtendedBlock blocki) throws IOException { - - if (!blocki.getBlockPoolId().equals(blockj.getBlockPoolId())) { - throw new IOException("Block pool IDs mismatched: block" + j + "=" - + blockj + ", block" + i + "=" + blocki); + if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) { + throw new IOException("Block pool IDs mismatched: block" + i + "=" + + blocki + ", expected block group=" + blockGroup); } - if (blocki.getBlockId() - i != blockj.getBlockId() - j) { - throw new IOException("Block IDs mismatched: block" + j + "=" - + blockj + ", block" + i + "=" + blocki); + if (blocki.getBlockId() - i != blockGroup.getBlockId()) { + throw new IOException("Block IDs mismatched: block" + i + "=" + + blocki + ", expected block group=" + blockGroup); } - if (blocki.getGenerationStamp() != blockj.getGenerationStamp()) { - throw new IOException("Generation stamps mismatched: block" + j + "=" - + blockj + ", block" + i + "=" + blocki); + if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) { + throw new IOException("Generation stamps mismatched: block" + i + "=" + + blocki + ", expected block group=" + blockGroup); } } 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 274d3195fb97c..e621f26d0e20f 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 @@ -1988,35 +1988,14 @@ public static Block addStripedBlockToFile(List dataNodes, */ public static ExtendedBlock flushInternal(DFSStripedOutputStream out) throws IOException { - out.flushInternal(); + out.flushAllInternals(); return out.getBlock(); } - /** - * Verify that blocks in striped block group are on different nodes, and every - * internal blocks exists. - */ - public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, - int groupSize) { - for (LocatedBlock lb : lbs.getLocatedBlocks()) { - assert lb instanceof LocatedStripedBlock; - HashSet locs = new HashSet<>(); - for (DatanodeInfo datanodeInfo : lb.getLocations()) { - locs.add(datanodeInfo); - } - assertEquals(groupSize, lb.getLocations().length); - assertEquals(groupSize, locs.size()); - - // verify that every internal blocks exists - int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices(); - assertEquals(groupSize, blockIndices.length); - HashSet found = new HashSet<>(); - for (int index : blockIndices) { - assert index >=0; - found.add(index); - } - assertEquals(groupSize, found.size()); - } + public static ExtendedBlock flushBuffer(DFSStripedOutputStream out) + throws IOException { + out.flush(); + return out.getBlock(); } public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java index 8d4a0cf5c9bf6..12453fafb3c9a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -20,23 +20,35 @@ import com.google.common.base.Joiner; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; 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.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.hdfs.web.ByteRangeInputStream; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.junit.Assert; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Random; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import static org.junit.Assert.assertEquals; + public class StripedFileTestUtil { public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class); /* @@ -50,8 +62,8 @@ public class StripedFileTestUtil { static final int stripesPerBlock = 4; static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock; static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2; + static final int BLOCK_GROUP_SIZE = blockSize * NUM_DATA_BLOCKS; - static final Random random = new Random(); static byte[] generateBytes(int cnt) { byte[] bytes = new byte[cnt]; @@ -61,6 +73,11 @@ static byte[] generateBytes(int cnt) { return bytes; } + static byte getByte(long pos) { + final int mod = 29; + return (byte) (pos % mod + 1); + } + static int readAll(FSDataInputStream in, byte[] buf) throws IOException { int readLen = 0; int ret; @@ -71,15 +88,10 @@ static int readAll(FSDataInputStream in, byte[] buf) throws IOException { return readLen; } - static byte getByte(long pos) { - final int mod = 29; - return (byte) (pos % mod + 1); - } - static void verifyLength(FileSystem fs, Path srcPath, int fileLength) throws IOException { FileStatus status = fs.getFileStatus(srcPath); - Assert.assertEquals("File length should be the same", fileLength, status.getLen()); + assertEquals("File length should be the same", fileLength, status.getLen()); } static void verifyPread(FileSystem fs, Path srcPath, int fileLength, @@ -101,9 +113,7 @@ static void verifyPread(FileSystem fs, Path srcPath, int fileLength, offset += target; } for (int i = 0; i < fileLength - startOffset; i++) { - Assert.assertEquals("Byte at " + (startOffset + i) + " is different, " - + "the startOffset is " + startOffset, - expected[startOffset + i], result[i]); + assertEquals("Byte at " + (startOffset + i) + " is different, " + "the startOffset is " + startOffset, expected[startOffset + i], result[i]); } } } @@ -119,8 +129,7 @@ static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, System.arraycopy(buf, 0, result, readLen, ret); readLen += ret; } - Assert.assertEquals("The length of file should be the same to write size", - fileLength, readLen); + assertEquals("The length of file should be the same to write size", fileLength, readLen); Assert.assertArrayEquals(expected, result); } } @@ -137,8 +146,7 @@ static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, result.put(buf); buf.clear(); } - Assert.assertEquals("The length of file should be the same to write size", - fileLength, readLen); + assertEquals("The length of file should be the same to write size", fileLength, readLen); Assert.assertArrayEquals(expected, result.array()); } } @@ -199,10 +207,9 @@ static void assertSeekAndRead(FSDataInputStream fsdis, int pos, fsdis.seek(pos); byte[] buf = new byte[writeBytes]; int readLen = StripedFileTestUtil.readAll(fsdis, buf); - Assert.assertEquals(readLen, writeBytes - pos); + assertEquals(readLen, writeBytes - pos); for (int i = 0; i < readLen; i++) { - Assert.assertEquals("Byte at " + i + " should be the same", - StripedFileTestUtil.getByte(pos + i), buf[i]); + assertEquals("Byte at " + i + " should be the same", StripedFileTestUtil.getByte(pos + i), buf[i]); } } @@ -210,6 +217,7 @@ static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, final int dnIndex, final AtomicInteger pos) { final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex); final DatanodeInfo datanode = getDatanodes(s); + assert datanode != null; LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos); cluster.stopDataNode(datanode.getXferAddr()); } @@ -218,7 +226,7 @@ static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { for(;;) { final DatanodeInfo[] datanodes = streamer.getNodes(); if (datanodes != null) { - Assert.assertEquals(1, datanodes.length); + assertEquals(1, datanodes.length); Assert.assertNotNull(datanodes[0]); return datanodes[0]; } @@ -287,7 +295,6 @@ public static void waitBlockGroupsReported(DistributedFileSystem fs, String src) * @param min minimum of the range * @param max maximum of the range * @param n number to be generated - * @return */ public static int[] randomArray(int min, int max, int n){ if (n > (max - min + 1) || max < min || min < 0 || max < 0) { @@ -315,4 +322,170 @@ public static int[] randomArray(int min, int max, int n){ } return result; } + + /** + * Verify that blocks in striped block group are on different nodes, and every + * internal blocks exists. + */ + public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, int groupSize) { + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + assert lb instanceof LocatedStripedBlock; + HashSet locs = new HashSet<>(); + Collections.addAll(locs, lb.getLocations()); + assertEquals(groupSize, lb.getLocations().length); + assertEquals(groupSize, locs.size()); + + // verify that every internal blocks exists + int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices(); + assertEquals(groupSize, blockIndices.length); + HashSet found = new HashSet<>(); + for (int index : blockIndices) { + assert index >=0; + found.add(index); + } + assertEquals(groupSize, found.size()); + } + } + + static void checkData(DistributedFileSystem dfs, Path srcPath, int length, + int[] killedDnIndex, long oldGS) throws IOException { + + StripedFileTestUtil.verifyLength(dfs, srcPath, length); + Arrays.sort(killedDnIndex); + List> blockGroupList = new ArrayList<>(); + LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L, + Long.MAX_VALUE); + int expectedNumGroup = 0; + if (length > 0) { + expectedNumGroup = (length - 1) / BLOCK_GROUP_SIZE + 1; + } + assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size()); + + for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { + Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); + + final long gs = firstBlock.getBlock().getGenerationStamp(); + final String s = "gs=" + gs + ", oldGS=" + oldGS; + LOG.info(s); + Assert.assertTrue(s, gs >= oldGS); + + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) firstBlock, BLOCK_STRIPED_CELL_SIZE, + NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + blockGroupList.add(Arrays.asList(blocks)); + } + + // test each block group + for (int group = 0; group < blockGroupList.size(); group++) { + final boolean isLastGroup = group == blockGroupList.size() - 1; + final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE + : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE; + final int numCellInGroup = (groupSize - 1)/BLOCK_STRIPED_CELL_SIZE + 1; + final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS; + final int lastCellSize = groupSize - (numCellInGroup - 1)*BLOCK_STRIPED_CELL_SIZE; + + //get the data of this block + List blockList = blockGroupList.get(group); + byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][]; + byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][]; + + // for each block, use BlockReader to read data + for (int i = 0; i < blockList.size(); i++) { + final int j = i >= NUM_DATA_BLOCKS? 0: i; + final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS + + (j <= lastCellIndex? 1: 0); + final int blockSize = numCellInBlock*BLOCK_STRIPED_CELL_SIZE + + (isLastGroup && j == lastCellIndex? lastCellSize - BLOCK_STRIPED_CELL_SIZE: 0); + + final byte[] blockBytes = new byte[blockSize]; + if (i < NUM_DATA_BLOCKS) { + dataBlockBytes[i] = blockBytes; + } else { + parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes; + } + + final LocatedBlock lb = blockList.get(i); + LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock + + ", blockSize=" + blockSize + ", lb=" + lb); + if (lb == null) { + continue; + } + final ExtendedBlock block = lb.getBlock(); + assertEquals(blockSize, block.getNumBytes()); + + if (block.getNumBytes() == 0) { + continue; + } + + if (Arrays.binarySearch(killedDnIndex, i) < 0) { + final BlockReader blockReader = BlockReaderTestUtil.getBlockReader( + dfs, lb, 0, block.getNumBytes()); + blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); + blockReader.close(); + } + } + + // check data + final int groupPosInFile = group*BLOCK_GROUP_SIZE; + for (int i = 0; i < dataBlockBytes.length; i++) { + boolean killed = false; + if (Arrays.binarySearch(killedDnIndex, i) >= 0){ + killed = true; + } + final byte[] actual = dataBlockBytes[i]; + for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) { + final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG( + BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile; + Assert.assertTrue(posInFile < length); + final byte expected = getByte(posInFile); + + if (killed) { + actual[posInBlk] = expected; + } else { + if(expected != actual[posInBlk]){ + String s = "expected=" + expected + " but actual=" + actual[posInBlk] + + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk + + ". group=" + group + ", i=" + i; + Assert.fail(s); + } + } + } + } + + // check parity + verifyParityBlocks(dfs.getConf(), lbs.getLocatedBlocks().get(group) + .getBlockSize(), + BLOCK_STRIPED_CELL_SIZE, dataBlockBytes, parityBlockBytes, killedDnIndex); + } + } + + static void verifyParityBlocks(Configuration conf, final long size, final int cellSize, + byte[][] dataBytes, byte[][] parityBytes, int[] killedDnIndex) { + Arrays.sort(killedDnIndex); + // verify the parity blocks + int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength( + size, cellSize, dataBytes.length, dataBytes.length); + final byte[][] expectedParityBytes = new byte[parityBytes.length][]; + for (int i = 0; i < parityBytes.length; i++) { + expectedParityBytes[i] = new byte[parityBlkSize]; + } + for (int i = 0; i < dataBytes.length; i++) { + if (dataBytes[i] == null) { + dataBytes[i] = new byte[dataBytes[0].length]; + } else if (dataBytes[i].length < dataBytes[0].length) { + final byte[] tmp = dataBytes[i]; + dataBytes[i] = new byte[dataBytes[0].length]; + System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length); + } + } + final RawErasureEncoder encoder = + CodecUtil.createRSRawEncoder(conf, dataBytes.length, parityBytes.length); + encoder.encode(dataBytes, expectedParityBytes); + for (int i = 0; i < parityBytes.length; i++) { + if (Arrays.binarySearch(killedDnIndex, dataBytes.length + i) < 0){ + Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + Arrays.toString(killedDnIndex), + expectedParityBytes[i], parityBytes[i]); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 0641e8ea65e94..d78e88b197220 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -18,26 +18,14 @@ package org.apache.hadoop.hdfs; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; import org.apache.hadoop.test.GenericTestUtils; import org.apache.log4j.Level; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -154,141 +142,15 @@ public void testFileMoreThanABlockGroup3() throws Exception { + cellSize + 123); } - private byte[] generateBytes(int cnt) { - byte[] bytes = new byte[cnt]; - for (int i = 0; i < cnt; i++) { - bytes[i] = getByte(i); - } - return bytes; - } - - private byte getByte(long pos) { - int mod = 29; - return (byte) (pos % mod + 1); - } - private void testOneFile(String src, int writeBytes) throws Exception { src += "_" + writeBytes; Path testPath = new Path(src); - byte[] bytes = generateBytes(writeBytes); + byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes); DFSTestUtil.writeFile(fs, testPath, new String(bytes)); StripedFileTestUtil.waitBlockGroupsReported(fs, src); - // check file length - FileStatus status = fs.getFileStatus(testPath); - Assert.assertEquals(writeBytes, status.getLen()); - - checkData(src, writeBytes); - } - - void checkData(String src, int writeBytes) throws IOException { - List> blockGroupList = new ArrayList<>(); - LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L); - - for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { - Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); - LocatedBlock[] blocks = StripedBlockUtil. - parseStripedBlockGroup((LocatedStripedBlock) firstBlock, - cellSize, dataBlocks, parityBlocks); - List oneGroup = Arrays.asList(blocks); - blockGroupList.add(oneGroup); - } - - // test each block group - for (int group = 0; group < blockGroupList.size(); group++) { - //get the data of this block - List blockList = blockGroupList.get(group); - byte[][] dataBlockBytes = new byte[dataBlocks][]; - byte[][] parityBlockBytes = new byte[parityBlocks][]; - - // for each block, use BlockReader to read data - for (int i = 0; i < blockList.size(); i++) { - LocatedBlock lblock = blockList.get(i); - if (lblock == null) { - continue; - } - ExtendedBlock block = lblock.getBlock(); - byte[] blockBytes = new byte[(int)block.getNumBytes()]; - if (i < dataBlocks) { - dataBlockBytes[i] = blockBytes; - } else { - parityBlockBytes[i - dataBlocks] = blockBytes; - } - - if (block.getNumBytes() == 0) { - continue; - } - - final BlockReader blockReader = BlockReaderTestUtil.getBlockReader( - fs, lblock, 0, block.getNumBytes()); - blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); - blockReader.close(); - } - - // check if we write the data correctly - for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length; - blkIdxInGroup++) { - final byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup]; - if (actualBlkBytes == null) { - continue; - } - for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) { - // calculate the position of this byte in the file - long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize, - dataBlocks, posInBlk, blkIdxInGroup) + - group * blockSize * dataBlocks; - Assert.assertTrue(posInFile < writeBytes); - final byte expected = getByte(posInFile); - - String s = "Unexpected byte " + actualBlkBytes[posInBlk] - + ", expect " + expected - + ". Block group index is " + group - + ", stripe index is " + posInBlk / cellSize - + ", cell index is " + blkIdxInGroup - + ", byte index is " + posInBlk % cellSize; - Assert.assertEquals(s, expected, actualBlkBytes[posInBlk]); - } - } - - verifyParity(lbs.getLocatedBlocks().get(group).getBlockSize(), - cellSize, dataBlockBytes, parityBlockBytes); - } - } - - void verifyParity(final long size, final int cellSize, - byte[][] dataBytes, byte[][] parityBytes) { - verifyParity(conf, size, cellSize, dataBytes, parityBytes, -1); - } - - static void verifyParity(Configuration conf, final long size, - final int cellSize, byte[][] dataBytes, - byte[][] parityBytes, int killedDnIndex) { - // verify the parity blocks - int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength( - size, cellSize, dataBytes.length, dataBytes.length); - final byte[][] expectedParityBytes = new byte[parityBytes.length][]; - for (int i = 0; i < parityBytes.length; i++) { - expectedParityBytes[i] = new byte[parityBlkSize]; - } - for (int i = 0; i < dataBytes.length; i++) { - if (dataBytes[i] == null) { - dataBytes[i] = new byte[dataBytes[0].length]; - } else if (dataBytes[i].length < dataBytes[0].length) { - final byte[] tmp = dataBytes[i]; - dataBytes[i] = new byte[dataBytes[0].length]; - System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length); - } - } - final RawErasureEncoder encoder = - CodecUtil.createRSRawEncoder(conf, - dataBytes.length, parityBytes.length); - encoder.encode(dataBytes, expectedParityBytes); - for (int i = 0; i < parityBytes.length; i++) { - if (i != killedDnIndex) { - Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + killedDnIndex, - expectedParityBytes[i], parityBytes[i]); - } - } + StripedFileTestUtil.checkData(fs, testPath, writeBytes, + new int[]{}, 0); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java index 44a29e671e5e6..f6c25661df68f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -30,23 +31,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; 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.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; 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.SecurityTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.namenode.NameNode; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; @@ -74,6 +70,7 @@ public class TestDFSStripedOutputStreamWithFailure { private static final int FLUSH_POS = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1; + static { System.out.println("NUM_DATA_BLOCKS = " + NUM_DATA_BLOCKS); System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS); @@ -101,6 +98,32 @@ static List newLengths() { return lengths; } + private static final int[][] dnIndexSuite = { + {0, 1}, + {0, 5}, + {0, 6}, + {0, 8}, + {1, 5}, + {1, 6}, + {6, 8}, + {0, 1, 2}, + {3, 4, 5}, + {0, 1, 6}, + {0, 5, 6}, + {0, 5, 8}, + {0, 6, 7}, + {5, 6, 7}, + {6, 7, 8}, + }; + + private int[] getKillPositions(int fileLen, int num) { + int[] positions = new int[num]; + for (int i = 0; i < num; i++) { + positions[i] = fileLen * (i + 1) / (num + 1); + } + return positions; + } + private static final List LENGTHS = newLengths(); static int getLength(int i) { @@ -127,41 +150,25 @@ private void tearDown() { } } - private static byte getByte(long pos) { - return (byte)pos; - } - private HdfsConfiguration newHdfsConfiguration() { final HdfsConfiguration conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); - conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L); + conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); return conf; } - void runTest(final int length) { - final HdfsConfiguration conf = newHdfsConfiguration(); - for (int dn = 0; dn < 9; dn++) { - try { - setup(conf); - runTest(length, dn, false, conf); - } catch (Exception e) { - final String err = "failed, dn=" + dn + ", length=" + length - + StringUtils.stringifyException(e); - LOG.error(err); - Assert.fail(err); - } finally { - tearDown(); - } - } - } - @Test(timeout=240000) public void testDatanodeFailure56() throws Exception { runTest(getLength(56)); } + @Test(timeout=240000) + public void testMultipleDatanodeFailure56() throws Exception { + runTestWithMultipleFailure(getLength(56)); + } + @Test(timeout=240000) public void testBlockTokenExpired() throws Exception { final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE); @@ -174,7 +181,7 @@ public void testBlockTokenExpired() throws Exception { for (int dn = 0; dn < 9; dn += 2) { try { setup(conf); - runTest(length, dn, true, conf); + runTest(length, new int[]{length/2}, new int[]{dn}, true); } catch (Exception e) { LOG.error("failed, dn=" + dn + ", length=" + length); throw e; @@ -214,22 +221,8 @@ public void testAddBlockWhenNoSufficientDataBlockNumOfNodes() Assert.fail("Failed to validate available dns against blkGroupSize"); } catch (IOException ioe) { // expected - GenericTestUtils.assertExceptionContains("Failed: the number of " - + "remaining blocks = 5 < the number of data blocks = 6", ioe); - DFSStripedOutputStream dfsout = (DFSStripedOutputStream) out - .getWrappedStream(); - - // get leading streamer and verify the last exception - StripedDataStreamer datastreamer = dfsout.getStripedDataStreamer(0); - try { - datastreamer.getLastException().check(true); - Assert.fail("Failed to validate available dns against blkGroupSize"); - } catch (IOException le) { - GenericTestUtils.assertExceptionContains( - "Failed to get datablocks number of nodes from" - + " namenode: blockGroupSize= 9, blocks.length= " - + numDatanodes, le); - } + GenericTestUtils.assertExceptionContains("Failed to get 6 nodes from" + + " namenode: blockGroupSize= 9, blocks.length= 5", ioe); } } finally { tearDown(); @@ -258,42 +251,73 @@ public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException { int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000; final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); DFSTestUtil.writeFile(dfs, srcPath, new String(expected)); + LOG.info("writing finished. Seek and read the file to verify."); StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength); } finally { tearDown(); } } - private void runTest(final int length, final int dnIndex, - final boolean tokenExpire, final HdfsConfiguration conf) { - try { - runTest(length, length/2, dnIndex, tokenExpire, conf); - } catch(Exception e) { - LOG.info("FAILED", e); - Assert.fail(StringUtils.stringifyException(e)); + void runTest(final int length) { + final HdfsConfiguration conf = newHdfsConfiguration(); + for (int dn = 0; dn < 9; dn++) { + try { + setup(conf); + runTest(length, new int[]{length/2}, new int[]{dn}, false); + } catch (Throwable e) { + final String err = "failed, dn=" + dn + ", length=" + length + + StringUtils.stringifyException(e); + LOG.error(err); + Assert.fail(err); + } finally { + tearDown(); + } } } - private void runTest(final int length, final int killPos, - final int dnIndex, final boolean tokenExpire, - final HdfsConfiguration conf) throws Exception { - if (killPos <= FLUSH_POS) { - LOG.warn("killPos=" + killPos + " <= FLUSH_POS=" + FLUSH_POS - + ", length=" + length + ", dnIndex=" + dnIndex); - return; //skip test + void runTestWithMultipleFailure(final int length) throws Exception { + final HdfsConfiguration conf = newHdfsConfiguration(); + for(int i=0;i> blockGroupList = new ArrayList<>(); - LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L); - final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1; - Assert.assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size()); - - for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { - Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); - - final long gs = firstBlock.getBlock().getGenerationStamp(); - final String s = "gs=" + gs + ", oldGS=" + oldGS; - LOG.info(s); - Assert.assertTrue(s, gs >= oldGS); - - LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( - (LocatedStripedBlock) firstBlock, - CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); - blockGroupList.add(Arrays.asList(blocks)); - } - - // test each block group - for (int group = 0; group < blockGroupList.size(); group++) { - final boolean isLastGroup = group == blockGroupList.size() - 1; - final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE - : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE; - final int numCellInGroup = (groupSize - 1)/CELL_SIZE + 1; - final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS; - final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE; - - //get the data of this block - List blockList = blockGroupList.get(group); - byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][]; - byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][]; - - // for each block, use BlockReader to read data - for (int i = 0; i < blockList.size(); i++) { - final int j = i >= NUM_DATA_BLOCKS? 0: i; - final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS - + (j <= lastCellIndex? 1: 0); - final int blockSize = numCellInBlock*CELL_SIZE - + (isLastGroup && j == lastCellIndex? lastCellSize - CELL_SIZE: 0); - - final byte[] blockBytes = new byte[blockSize]; - if (i < NUM_DATA_BLOCKS) { - dataBlockBytes[i] = blockBytes; - } else { - parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes; - } - - final LocatedBlock lb = blockList.get(i); - LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock - + ", blockSize=" + blockSize + ", lb=" + lb); - if (lb == null) { - continue; - } - final ExtendedBlock block = lb.getBlock(); - Assert.assertEquals(blockSize, block.getNumBytes()); - - - if (block.getNumBytes() == 0) { - continue; - } - - if (i != killedDnIndex) { - final BlockReader blockReader = BlockReaderTestUtil.getBlockReader( - dfs, lb, 0, block.getNumBytes()); - blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); - blockReader.close(); - } - } - - // check data - final int groupPosInFile = group*BLOCK_GROUP_SIZE; - for (int i = 0; i < dataBlockBytes.length; i++) { - final byte[] actual = dataBlockBytes[i]; - for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) { - final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG( - CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile; - Assert.assertTrue(posInFile < length); - final byte expected = getByte(posInFile); - - if (i == killedDnIndex) { - actual[posInBlk] = expected; - } else { - String s = "expected=" + expected + " but actual=" + actual[posInBlk] - + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk - + ". group=" + group + ", i=" + i; - Assert.assertEquals(s, expected, actual[posInBlk]); - } - } - } - - // check parity - TestDFSStripedOutputStream.verifyParity(dfs.getConf(), - lbs.getLocatedBlocks().get(group).getBlockSize(), - CELL_SIZE, dataBlockBytes, parityBlockBytes, - killedDnIndex - dataBlockBytes.length); - } - } private void waitTokenExpires(FSDataOutputStream out) throws IOException { Token token = DFSTestUtil.getBlockToken(out); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java index c0dca4e8662a9..764527d907429 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -23,6 +23,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; @@ -39,6 +41,12 @@ public class TestWriteStripedFileWithFailure { private static MiniDFSCluster cluster; private static FileSystem fs; private static Configuration conf = new HdfsConfiguration(); + + static { + GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); + GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); + } + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; private final int smallFileLength = blockSize * dataBlocks - 123; 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 124bf8060eda3..ef315275f979a 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 @@ -1745,7 +1745,7 @@ private void doTestBalancerWithStripedFile(Configuration conf) throws Exception // verify locations of striped blocks LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen); - DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); + StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); // add one datanode String newRack = "/rack" + (++numOfRacks); @@ -1761,7 +1761,7 @@ private void doTestBalancerWithStripedFile(Configuration conf) throws Exception // verify locations of striped blocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen); - DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); + StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); } finally { cluster.shutdown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index 3a9748f223942..7cf56562a5972 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -488,7 +488,7 @@ public void testMoverWithStripedFile() throws Exception { Assert.assertEquals(StorageType.DISK, type); } } - DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, + StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, dataBlocks + parityBlocks); // start 5 more datanodes @@ -523,7 +523,7 @@ public void testMoverWithStripedFile() throws Exception { Assert.assertEquals(StorageType.ARCHIVE, type); } } - DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, + StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, dataBlocks + parityBlocks); }finally{ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java index 64d33a406fa82..abcdbc10c0847 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.StripedFileTestUtil; 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.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; @@ -42,7 +41,6 @@ import java.util.List; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; public class TestAddOverReplicatedStripedBlocks { @@ -64,6 +62,7 @@ public void setup() throws IOException { conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); // disable block recovery conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); SimulatedFSDataset.setFactory(conf); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster.waitActive(); @@ -118,7 +117,7 @@ public void testProcessOverReplicatedStripedBlock() throws Exception { // verify that all internal blocks exists lbs = cluster.getNameNodeRpc().getBlockLocations( filePath.toString(), 0, fileLen); - DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); } @Test @@ -162,7 +161,7 @@ public void testProcessOverReplicatedSBSmallerThanFullBlocks() // verify that all internal blocks exists lbs = cluster.getNameNodeRpc().getBlockLocations( filePath.toString(), 0, fileLen); - DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); } @Test @@ -216,7 +215,7 @@ public void testProcessOverReplicatedAndCorruptStripedBlock() // verify that all internal blocks exists lbs = cluster.getNameNodeRpc().getBlockLocations( filePath.toString(), 0, fileLen); - DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); } @Test @@ -248,6 +247,7 @@ public void testProcessOverReplicatedAndMissingStripedBlock() // update blocksMap cluster.triggerBlockReports(); + Thread.sleep(2000); // add to invalidates cluster.triggerHeartbeats(); // datanode delete block @@ -259,7 +259,7 @@ public void testProcessOverReplicatedAndMissingStripedBlock() // we are left GROUP_SIZE - 1 blocks. lbs = cluster.getNameNodeRpc().getBlockLocations( filePath.toString(), 0, fileLen); - DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); } } 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 c27ead5f659e8..735f84dfa12cd 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 @@ -73,6 +73,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper; @@ -736,7 +737,13 @@ void invoke() throws Exception { DatanodeInfo[] newNodes = new DatanodeInfo[2]; newNodes[0] = nodes[0]; newNodes[1] = nodes[1]; - String[] storageIDs = {"s0", "s1"}; + final DatanodeManager dm = cluster.getNamesystem(0).getBlockManager() + .getDatanodeManager(); + final String storageID1 = dm.getDatanode(newNodes[0]).getStorageInfos()[0] + .getStorageID(); + final String storageID2 = dm.getDatanode(newNodes[1]).getStorageInfos()[0] + .getStorageID(); + String[] storageIDs = {storageID1, storageID2}; client.getNamenode().updatePipeline(client.getClientName(), oldBlock, newBlock, newNodes, storageIDs);