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 extends RawErasureEncoder> encoderClass;
+ protected Class extends RawErasureDecoder> 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 extends ErasureEncoder> encoderClass;
+ protected Class extends ErasureDecoder> 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 extends ErasureEncoder> encoderClass;
protected Class extends ErasureDecoder> 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 extends BlockInfo> 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 extends Block> 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 extends ErasureEncoder> encoderClass;
protected Class extends ErasureDecoder> 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 extends RawErasureCoderFactory> 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 extends Block> 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 extends Block> 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 extends BlockInfo> 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