diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 9e3e826ea77..bace31168b4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -177,6 +177,12 @@ Trunk (Unreleased)
HADOOP-10824. Refactor KMSACLs to avoid locking. (Benoy Antony via umamahesh)
+ HADOOP-10841. EncryptedKeyVersion should have a key name property.
+ (asuresh via tucu)
+
+ HADOOP-10842. CryptoExtension generateEncryptedKey method should
+ receive the key name. (asuresh via tucu)
+
BUG FIXES
HADOOP-9451. Fault single-layer config if node group topology is enabled.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 4a802ed8f52..204af819a4c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -44,17 +44,23 @@ public class KeyProviderCryptoExtension extends
* used to generate the encrypted Key and the encrypted KeyVersion
*/
public static class EncryptedKeyVersion {
+ private String keyName;
private String keyVersionName;
private byte[] iv;
private KeyVersion encryptedKey;
- protected EncryptedKeyVersion(String keyVersionName, byte[] iv,
- KeyVersion encryptedKey) {
+ protected EncryptedKeyVersion(String keyName, String keyVersionName,
+ byte[] iv, KeyVersion encryptedKey) {
+ this.keyName = keyName;
this.keyVersionName = keyVersionName;
this.iv = iv;
this.encryptedKey = encryptedKey;
}
+ public String getKeyName() {
+ return keyName;
+ }
+
public String getKeyVersionName() {
return keyVersionName;
}
@@ -78,14 +84,13 @@ public class KeyProviderCryptoExtension extends
/**
* Generates a key material and encrypts it using the given key version name
* and initialization vector. The generated key material is of the same
- * length as the KeyVersion
material and is encrypted using the
- * same cipher.
+ * length as the KeyVersion
material of the latest key version
+ * of the key and is encrypted using the same cipher.
*
* NOTE: The generated key is not stored by the KeyProvider
*
- * @param encryptionKeyVersion
- * a KeyVersion object containing the keyVersion name and material
- * to encrypt.
+ * @param encryptionKeyName
+ * The latest KeyVersion of this key's material will be encrypted.
* @return EncryptedKeyVersion with the generated key material, the version
* name is 'EEK' (for Encrypted Encryption Key)
* @throws IOException
@@ -95,7 +100,7 @@ public class KeyProviderCryptoExtension extends
* cryptographic issue.
*/
public EncryptedKeyVersion generateEncryptedKey(
- KeyVersion encryptionKeyVersion) throws IOException,
+ String encryptionKeyName) throws IOException,
GeneralSecurityException;
/**
@@ -140,12 +145,11 @@ public class KeyProviderCryptoExtension extends
}
@Override
- public EncryptedKeyVersion generateEncryptedKey(KeyVersion keyVersion)
+ public EncryptedKeyVersion generateEncryptedKey(String encryptionKeyName)
throws IOException, GeneralSecurityException {
- KeyVersion keyVer =
- keyProvider.getKeyVersion(keyVersion.getVersionName());
- Preconditions.checkNotNull(keyVer, "KeyVersion name '%s' does not exist",
- keyVersion.getVersionName());
+ KeyVersion keyVer = keyProvider.getCurrentKey(encryptionKeyName);
+ Preconditions.checkNotNull(keyVer, "No KeyVersion exists for key '%s' ",
+ encryptionKeyName);
byte[] newKey = new byte[keyVer.getMaterial().length];
SecureRandom.getInstance("SHA1PRNG").nextBytes(newKey);
Cipher cipher = Cipher.getInstance("AES/CTR/NoPadding");
@@ -153,7 +157,8 @@ public class KeyProviderCryptoExtension extends
cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(keyVer.getMaterial(),
"AES"), new IvParameterSpec(flipIV(iv)));
byte[] ek = cipher.doFinal(newKey);
- return new EncryptedKeyVersion(keyVersion.getVersionName(), iv,
+ return new EncryptedKeyVersion(encryptionKeyName,
+ keyVer.getVersionName(), iv,
new KeyVersion(keyVer.getName(), EEK, ek));
}
@@ -190,18 +195,18 @@ public class KeyProviderCryptoExtension extends
*
* NOTE: The generated key is not stored by the KeyProvider
*
- * @param encryptionKey a KeyVersion object containing the keyVersion name and
- * material to encrypt.
+ * @param encryptionKeyName The latest KeyVersion of this key's material will
+ * be encrypted.
* @return EncryptedKeyVersion with the generated key material, the version
* name is 'EEK' (for Encrypted Encryption Key)
* @throws IOException thrown if the key material could not be generated
* @throws GeneralSecurityException thrown if the key material could not be
* encrypted because of a cryptographic issue.
*/
- public EncryptedKeyVersion generateEncryptedKey(KeyVersion encryptionKey)
+ public EncryptedKeyVersion generateEncryptedKey(String encryptionKeyName)
throws IOException,
GeneralSecurityException {
- return getExtension().generateEncryptedKey(encryptionKey);
+ return getExtension().generateEncryptedKey(encryptionKeyName);
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
index d8da557d40e..4b578c23775 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
@@ -42,9 +42,10 @@ public class TestKeyProviderCryptoExtension {
KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
KeyProviderCryptoExtension.EncryptedKeyVersion ek1 =
- kpExt.generateEncryptedKey(kv);
+ kpExt.generateEncryptedKey(kv.getName());
Assert.assertEquals(KeyProviderCryptoExtension.EEK,
ek1.getEncryptedKey().getVersionName());
+ Assert.assertEquals("foo", ek1.getKeyName());
Assert.assertNotNull(ek1.getEncryptedKey().getMaterial());
Assert.assertEquals(kv.getMaterial().length,
ek1.getEncryptedKey().getMaterial().length);
@@ -55,7 +56,7 @@ public class TestKeyProviderCryptoExtension {
Assert.assertEquals(kv.getMaterial().length, k1.getMaterial().length);
KeyProviderCryptoExtension.EncryptedKeyVersion ek2 =
- kpExt.generateEncryptedKey(kv);
+ kpExt.generateEncryptedKey(kv.getName());
KeyProvider.KeyVersion k2 = kpExt.decryptEncryptedKey(ek2);
boolean eq = true;
for (int i = 0; eq && i < ek2.getEncryptedKey().getMaterial().length; i++) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6bd8392539b..2afe1f6d8ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -303,6 +303,18 @@ Release 2.6.0 - UNRELEASED
HDFS-5624. Add HDFS tests for ACLs in combination with viewfs.
(Stephen Chu via cnauroth)
+ HDFS-6655. Add 'header banner' to 'explorer.html' also in Namenode UI
+ (vinayakumarb)
+
+ HDFS-4120. Add a new "-skipSharedEditsCheck" option for BootstrapStandby
+ (Liang Xie and Rakesh R via vinayakumarb)
+
+ HDFS-6597. Add a new option to NN upgrade to terminate the process after
+ upgrade on NN is completed. (Danilo Vunjak via cnauroth)
+
+ HDFS-6700. BlockPlacementPolicy shoud choose storage but not datanode for
+ deletion. (szetszwo)
+
OPTIMIZATIONS
HDFS-6690. Deduplicate xattr names in memory. (wang)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java
new file mode 100644
index 00000000000..ded9e0e99cb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java
@@ -0,0 +1,169 @@
+/**
+ * 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.contrib.bkjournal;
+
+import java.io.File;
+import java.io.FileFilter;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints.SlowCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+
+public class TestBootstrapStandbyWithBKJM {
+ private static BKJMUtil bkutil;
+ protected MiniDFSCluster cluster;
+
+ @BeforeClass
+ public static void setupBookkeeper() throws Exception {
+ bkutil = new BKJMUtil(3);
+ bkutil.start();
+ }
+
+ @AfterClass
+ public static void teardownBookkeeper() throws Exception {
+ bkutil.teardown();
+ }
+
+ @After
+ public void teardown() {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
+ conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+ conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
+ .createJournalURI("/bootstrapStandby").toString());
+ BKJMUtil.addJournalManagerDefinition(conf);
+ conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
+ conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
+ SlowCodec.class.getCanonicalName());
+ CompressionCodecFactory.setCodecClasses(conf,
+ ImmutableList. of(SlowCodec.class));
+ MiniDFSNNTopology topology = new MiniDFSNNTopology()
+ .addNameservice(new MiniDFSNNTopology.NSConf("ns1").addNN(
+ new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)).addNN(
+ new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+ cluster = new MiniDFSCluster.Builder(conf).nnTopology(topology)
+ .numDataNodes(1).manageNameDfsSharedDirs(false).build();
+ cluster.waitActive();
+ }
+
+ /**
+ * While boostrapping, in_progress transaction entries should be skipped.
+ * Bootstrap usage for BKJM : "-force", "-nonInteractive", "-skipSharedEditsCheck"
+ */
+ @Test
+ public void testBootstrapStandbyWithActiveNN() throws Exception {
+ // make nn0 active
+ cluster.transitionToActive(0);
+
+ // do ops and generate in-progress edit log data
+ Configuration confNN1 = cluster.getConfiguration(1);
+ DistributedFileSystem dfs = (DistributedFileSystem) HATestUtil
+ .configureFailoverFs(cluster, confNN1);
+ for (int i = 1; i <= 10; i++) {
+ dfs.mkdirs(new Path("/test" + i));
+ }
+ dfs.close();
+
+ // shutdown nn1 and delete its edit log files
+ cluster.shutdownNameNode(1);
+ deleteEditLogIfExists(confNN1);
+ cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_ENTER, true);
+ cluster.getNameNodeRpc(0).saveNamespace();
+ cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_LEAVE, true);
+
+ // check without -skipSharedEditsCheck, Bootstrap should fail for BKJM
+ // immediately after saveNamespace
+ int rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive" },
+ confNN1);
+ Assert.assertEquals("Mismatches return code", 6, rc);
+
+ // check with -skipSharedEditsCheck
+ rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive",
+ "-skipSharedEditsCheck" }, confNN1);
+ Assert.assertEquals("Mismatches return code", 0, rc);
+
+ // Checkpoint as fast as we can, in a tight loop.
+ confNN1.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 1);
+ cluster.restartNameNode(1);
+ cluster.transitionToStandby(1);
+
+ NameNode nn0 = cluster.getNameNode(0);
+ HATestUtil.waitForStandbyToCatchUp(nn0, cluster.getNameNode(1));
+ long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
+ .getFSImage().getMostRecentCheckpointTxId();
+ HATestUtil.waitForCheckpoint(cluster, 1,
+ ImmutableList.of((int) expectedCheckpointTxId));
+
+ // Should have copied over the namespace
+ FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
+ ImmutableList.of((int) expectedCheckpointTxId));
+ FSImageTestUtil.assertNNFilesMatch(cluster);
+ }
+
+ private void deleteEditLogIfExists(Configuration confNN1) {
+ String editDirs = confNN1.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
+ String[] listEditDirs = StringUtils.split(editDirs, ',');
+ Assert.assertTrue("Wrong edit directory path!", listEditDirs.length > 0);
+
+ for (String dir : listEditDirs) {
+ File curDir = new File(dir, "current");
+ File[] listFiles = curDir.listFiles(new FileFilter() {
+ @Override
+ public boolean accept(File f) {
+ if (!f.getName().startsWith("edits")) {
+ return true;
+ }
+ return false;
+ }
+ });
+ if (listFiles != null && listFiles.length > 0) {
+ for (File file : listFiles) {
+ Assert.assertTrue("Failed to delete edit files!", file.delete());
+ }
+ }
+ }
+ }
+}
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 999612ebabf..dc322b28595 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
@@ -2660,7 +2660,7 @@ public class BlockManager {
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)) {
@@ -2680,7 +2680,7 @@ public class BlockManager {
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
// exclude corrupt replicas
if (corruptNodes == null || !corruptNodes.contains(cur)) {
- nonExcess.add(cur);
+ nonExcess.add(storage);
}
}
}
@@ -2704,7 +2704,7 @@ public class BlockManager {
* If no such a node is available,
* then pick a node with least free space
*/
- private void chooseExcessReplicates(Collection nonExcess,
+ private void chooseExcessReplicates(final Collection nonExcess,
Block b, short replication,
DatanodeDescriptor addedNode,
DatanodeDescriptor delNodeHint,
@@ -2712,28 +2712,33 @@ public class BlockManager {
assert namesystem.hasWriteLock();
// first form a rack to datanodes map and
BlockCollection bc = getBlockCollection(b);
- 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
// exactlyOne contains the remaining nodes
- replicator.splitNodesWithRack(nonExcess, rackMap, moreThanOne,
- exactlyOne);
+ replicator.splitNodesWithRack(nonExcess, rackMap, moreThanOne, exactlyOne);
// pick one node to delete that favors the delete hint
// otherwise pick one with least space from priSet if it is not empty
// otherwise one node with least space from remains
boolean firstOne = true;
+ final DatanodeStorageInfo delNodeHintStorage
+ = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, delNodeHint);
+ final DatanodeStorageInfo addedNodeStorage
+ = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, addedNode);
while (nonExcess.size() - replication > 0) {
// check if we can delete delNodeHint
- final DatanodeInfo cur;
- if (firstOne && delNodeHint !=null && nonExcess.contains(delNodeHint)
- && (moreThanOne.contains(delNodeHint)
- || (addedNode != null && !moreThanOne.contains(addedNode))) ) {
- cur = delNodeHint;
+ final DatanodeStorageInfo cur;
+ if (firstOne && delNodeHintStorage != null
+ && (moreThanOne.contains(delNodeHintStorage)
+ || (addedNodeStorage != null
+ && !moreThanOne.contains(addedNodeStorage)))) {
+ cur = delNodeHintStorage;
} else { // regular excessive replica removal
cur = replicator.chooseReplicaToDelete(bc, b, replication,
moreThanOne, exactlyOne);
@@ -2745,7 +2750,7 @@ public class BlockManager {
exactlyOne, cur);
nonExcess.remove(cur);
- addToExcessReplicate(cur, b);
+ addToExcessReplicate(cur.getDatanodeDescriptor(), b);
//
// The 'excessblocks' tracks blocks until we get confirmation
@@ -2756,7 +2761,7 @@ public class BlockManager {
// should be deleted. Items are removed from the invalidate list
// upon giving instructions to the namenode.
//
- addToInvalidates(b, cur);
+ addToInvalidates(b, cur.getDatanodeDescriptor());
blockLog.info("BLOCK* chooseExcessReplicates: "
+"("+cur+", "+b+") is added to invalidated blocks set");
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
index 7b64abd5c17..8c6f57c1998 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
@@ -125,11 +125,12 @@ public abstract class BlockPlacementPolicy {
listed in the previous parameter.
* @return the replica that is the best candidate for deletion
*/
- abstract public DatanodeDescriptor chooseReplicaToDelete(BlockCollection srcBC,
- Block block,
- short replicationFactor,
- Collection existingReplicas,
- Collection moreExistingReplicas);
+ abstract public DatanodeStorageInfo chooseReplicaToDelete(
+ BlockCollection srcBC,
+ Block block,
+ short replicationFactor,
+ Collection existingReplicas,
+ Collection moreExistingReplicas);
/**
* Used to setup a BlockPlacementPolicy object. This should be defined by
@@ -176,21 +177,23 @@ public abstract class BlockPlacementPolicy {
* @param exactlyOne The List of replica nodes on rack with only one replica
* @param cur current replica to remove
*/
- public void adjustSetsWithChosenReplica(final Map> rackMap,
- final List moreThanOne,
- final List exactlyOne, final DatanodeInfo cur) {
+ public void adjustSetsWithChosenReplica(
+ final Map> rackMap,
+ final List moreThanOne,
+ final List exactlyOne,
+ final DatanodeStorageInfo cur) {
- String rack = getRack(cur);
- final List datanodes = rackMap.get(rack);
- datanodes.remove(cur);
- if (datanodes.isEmpty()) {
+ final String rack = getRack(cur.getDatanodeDescriptor());
+ final List storages = rackMap.get(rack);
+ storages.remove(cur);
+ if (storages.isEmpty()) {
rackMap.remove(rack);
}
if (moreThanOne.remove(cur)) {
- if (datanodes.size() == 1) {
- moreThanOne.remove(datanodes.get(0));
- exactlyOne.add(datanodes.get(0));
+ if (storages.size() == 1) {
+ final DatanodeStorageInfo remaining = storages.get(0);
+ moreThanOne.remove(remaining);
+ exactlyOne.add(remaining);
}
} else {
exactlyOne.remove(cur);
@@ -215,28 +218,28 @@ public abstract class BlockPlacementPolicy {
* @param exactlyOne remains contains the remaining nodes
*/
public void splitNodesWithRack(
- Collection dataNodes,
- final Map> rackMap,
- final List moreThanOne,
- final List exactlyOne) {
- for(DatanodeDescriptor node : dataNodes) {
- final String rackName = getRack(node);
- List datanodeList = rackMap.get(rackName);
- if (datanodeList == null) {
- datanodeList = new ArrayList();
- rackMap.put(rackName, datanodeList);
+ final Iterable storages,
+ final Map> rackMap,
+ final List moreThanOne,
+ final List exactlyOne) {
+ for(DatanodeStorageInfo s: storages) {
+ final String rackName = getRack(s.getDatanodeDescriptor());
+ List storageList = rackMap.get(rackName);
+ if (storageList == null) {
+ storageList = new ArrayList();
+ rackMap.put(rackName, storageList);
}
- datanodeList.add(node);
+ storageList.add(s);
}
// split nodes into two sets
- for(List datanodeList : rackMap.values()) {
- if (datanodeList.size() == 1) {
+ for(List storageList : rackMap.values()) {
+ if (storageList.size() == 1) {
// exactlyOne contains nodes on rack with only one replica
- exactlyOne.add(datanodeList.get(0));
+ exactlyOne.add(storageList.get(0));
} else {
// moreThanOne contains nodes on rack with more than one replica
- moreThanOne.addAll(datanodeList);
+ moreThanOne.addAll(storageList);
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index 33a5c5478fe..80e40cde898 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -756,31 +756,34 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
}
@Override
- public DatanodeDescriptor chooseReplicaToDelete(BlockCollection bc,
+ public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection bc,
Block block, short replicationFactor,
- Collection first,
- Collection second) {
+ Collection first,
+ Collection second) {
long oldestHeartbeat =
now() - heartbeatInterval * tolerateHeartbeatMultiplier;
- DatanodeDescriptor oldestHeartbeatNode = null;
+ DatanodeStorageInfo oldestHeartbeatStorage = null;
long minSpace = Long.MAX_VALUE;
- DatanodeDescriptor minSpaceNode = null;
+ DatanodeStorageInfo minSpaceStorage = null;
// Pick the node with the oldest heartbeat or with the least free space,
// if all hearbeats are within the tolerable heartbeat interval
- for(DatanodeDescriptor node : pickupReplicaSet(first, second)) {
+ for(DatanodeStorageInfo storage : pickupReplicaSet(first, second)) {
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
long free = node.getRemaining();
long lastHeartbeat = node.getLastUpdate();
if(lastHeartbeat < oldestHeartbeat) {
oldestHeartbeat = lastHeartbeat;
- oldestHeartbeatNode = node;
+ oldestHeartbeatStorage = storage;
}
if (minSpace > free) {
minSpace = free;
- minSpaceNode = node;
+ minSpaceStorage = storage;
}
}
- return oldestHeartbeatNode != null ? oldestHeartbeatNode : minSpaceNode;
+
+ return oldestHeartbeatStorage != null? oldestHeartbeatStorage
+ : minSpaceStorage;
}
/**
@@ -789,9 +792,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
* replica while second set contains remaining replica nodes.
* So pick up first set if not empty. If first is empty, then pick second.
*/
- protected Collection pickupReplicaSet(
- Collection first,
- Collection second) {
+ protected Collection pickupReplicaSet(
+ Collection first,
+ Collection second) {
return first.isEmpty() ? second : first;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
index c6a85a0d331..1069b4e3d75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
@@ -286,9 +286,9 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
* If first is empty, then pick second.
*/
@Override
- public Collection pickupReplicaSet(
- Collection first,
- Collection second) {
+ public Collection pickupReplicaSet(
+ Collection first,
+ Collection second) {
// If no replica within same rack, return directly.
if (first.isEmpty()) {
return second;
@@ -296,25 +296,24 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
// Split data nodes in the first set into two sets,
// moreThanOne contains nodes on nodegroup with more than one replica
// exactlyOne contains the remaining nodes
- Map> nodeGroupMap =
- new HashMap>();
+ Map> nodeGroupMap =
+ new HashMap>();
- for(DatanodeDescriptor node : first) {
- final String nodeGroupName =
- NetworkTopology.getLastHalf(node.getNetworkLocation());
- List datanodeList =
- nodeGroupMap.get(nodeGroupName);
- if (datanodeList == null) {
- datanodeList = new ArrayList();
- nodeGroupMap.put(nodeGroupName, datanodeList);
+ for(DatanodeStorageInfo storage : first) {
+ final String nodeGroupName = NetworkTopology.getLastHalf(
+ storage.getDatanodeDescriptor().getNetworkLocation());
+ List storageList = nodeGroupMap.get(nodeGroupName);
+ if (storageList == null) {
+ storageList = new ArrayList();
+ nodeGroupMap.put(nodeGroupName, storageList);
}
- datanodeList.add(node);
+ storageList.add(storage);
}
- final List moreThanOne = new ArrayList();
- final List exactlyOne = new ArrayList();
+ final List moreThanOne = new ArrayList();
+ final List exactlyOne = new ArrayList();
// split nodes into two sets
- for(List datanodeList : nodeGroupMap.values()) {
+ for(List datanodeList : nodeGroupMap.values()) {
if (datanodeList.size() == 1 ) {
// exactlyOne contains nodes on nodegroup with exactly one replica
exactlyOne.add(datanodeList.get(0));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index c1788910f77..f7bab3ced6a 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
@@ -22,6 +22,7 @@ import java.util.Iterator;
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
+
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -290,4 +291,21 @@ public class DatanodeStorageInfo {
public String toString() {
return "[" + storageType + "]" + storageID + ":" + state;
}
+
+ /** @return the first {@link DatanodeStorageInfo} corresponding to
+ * the given datanode
+ */
+ static DatanodeStorageInfo getDatanodeStorageInfo(
+ final Iterable infos,
+ final DatanodeDescriptor datanode) {
+ if (datanode == null) {
+ return null;
+ }
+ for(DatanodeStorageInfo storage : infos) {
+ if (storage.getDatanodeDescriptor() == datanode) {
+ return storage;
+ }
+ }
+ return null;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 486d426bab1..8c63cc47a48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -93,7 +93,8 @@ public final class HdfsServerConstants {
FORCE("-force"),
NONINTERACTIVE("-nonInteractive"),
RENAMERESERVED("-renameReserved"),
- METADATAVERSION("-metadataVersion");
+ METADATAVERSION("-metadataVersion"),
+ UPGRADEONLY("-upgradeOnly");
private static final Pattern ENUM_WITH_ROLLING_UPGRADE_OPTION = Pattern.compile(
"(\\w+)\\((\\w+)\\)");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 7a24a52e8cc..73101a81c37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -225,6 +225,7 @@ public class FSImage implements Closeable {
NNStorage.checkVersionUpgradable(storage.getLayoutVersion());
}
if (startOpt != StartupOption.UPGRADE
+ && startOpt != StartupOption.UPGRADEONLY
&& !RollingUpgradeStartupOption.STARTED.matches(startOpt)
&& layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
&& layoutVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
@@ -263,6 +264,7 @@ public class FSImage implements Closeable {
// 3. Do transitions
switch(startOpt) {
case UPGRADE:
+ case UPGRADEONLY:
doUpgrade(target);
return false; // upgrade saved image already
case IMPORT:
@@ -748,11 +750,13 @@ public class FSImage implements Closeable {
editLog.recoverUnclosedStreams();
} else if (HAUtil.isHAEnabled(conf, nameserviceId)
&& (startOpt == StartupOption.UPGRADE
+ || startOpt == StartupOption.UPGRADEONLY
|| RollingUpgradeStartupOption.ROLLBACK.matches(startOpt))) {
// This NN is HA, but we're doing an upgrade or a rollback of rolling
// upgrade so init the edit log for write.
editLog.initJournalsForWrite();
- if (startOpt == StartupOption.UPGRADE) {
+ if (startOpt == StartupOption.UPGRADE
+ || startOpt == StartupOption.UPGRADEONLY) {
long sharedLogCTime = editLog.getSharedLogCTime();
if (this.storage.getCTime() < sharedLogCTime) {
throw new IOException("It looks like the shared log is already " +
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 49a000c9ddd..53f8a18ec57 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
@@ -978,7 +978,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
// This will start a new log segment and write to the seen_txid file, so
// we shouldn't do it when coming up in standby state
- if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)) {
+ if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)
+ || (haEnabled && startOpt == StartupOption.UPGRADEONLY)) {
fsImage.openEditLogForWrite();
}
success = true;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index a576f1c5bb1..2eef037ae8d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -836,7 +836,7 @@ public class NNStorage extends Storage implements Closeable,
*/
void processStartupOptionsForUpgrade(StartupOption startOpt, int layoutVersion)
throws IOException {
- if (startOpt == StartupOption.UPGRADE) {
+ if (startOpt == StartupOption.UPGRADE || startOpt == StartupOption.UPGRADEONLY) {
// If upgrade from a release that does not support federation,
// if clusterId is provided in the startupOptions use it.
// Else generate a new cluster ID
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 645b5997925..4072b1720d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -210,6 +210,9 @@ public class NameNode implements NameNodeStatusMXBean {
+ StartupOption.UPGRADE.getName() +
" [" + StartupOption.CLUSTERID.getName() + " cid]" +
" [" + StartupOption.RENAMERESERVED.getName() + "] ] | \n\t["
+ + StartupOption.UPGRADEONLY.getName() +
+ " [" + StartupOption.CLUSTERID.getName() + " cid]" +
+ " [" + StartupOption.RENAMERESERVED.getName() + "] ] | \n\t["
+ StartupOption.ROLLBACK.getName() + "] | \n\t["
+ StartupOption.ROLLINGUPGRADE.getName() + " <"
+ RollingUpgradeStartupOption.DOWNGRADE.name().toLowerCase() + "|"
@@ -713,6 +716,7 @@ public class NameNode implements NameNodeStatusMXBean {
* {@link StartupOption#BACKUP BACKUP} - start backup node
* {@link StartupOption#CHECKPOINT CHECKPOINT} - start checkpoint node
* {@link StartupOption#UPGRADE UPGRADE} - start the cluster
+ * {@link StartupOption#UPGRADEONLY UPGRADEONLY} - upgrade the cluster
* upgrade and create a snapshot of the current file system state
* {@link StartupOption#RECOVER RECOVERY} - recover name node
* metadata
@@ -767,7 +771,8 @@ public class NameNode implements NameNodeStatusMXBean {
}
protected HAState createHAState(StartupOption startOpt) {
- if (!haEnabled || startOpt == StartupOption.UPGRADE) {
+ if (!haEnabled || startOpt == StartupOption.UPGRADE
+ || startOpt == StartupOption.UPGRADEONLY) {
return ACTIVE_STATE;
} else {
return STANDBY_STATE;
@@ -1198,8 +1203,10 @@ public class NameNode implements NameNodeStatusMXBean {
startOpt = StartupOption.BACKUP;
} else if (StartupOption.CHECKPOINT.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.CHECKPOINT;
- } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) {
- startOpt = StartupOption.UPGRADE;
+ } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)
+ || StartupOption.UPGRADEONLY.getName().equalsIgnoreCase(cmd)) {
+ startOpt = StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd) ?
+ StartupOption.UPGRADE : StartupOption.UPGRADEONLY;
/* Can be followed by CLUSTERID with a required parameter or
* RENAMERESERVED with an optional parameter
*/
@@ -1407,6 +1414,12 @@ public class NameNode implements NameNodeStatusMXBean {
terminate(0);
return null; // avoid javac warning
}
+ case UPGRADEONLY: {
+ DefaultMetricsSystem.initialize("NameNode");
+ new NameNode(conf);
+ terminate(0);
+ return null;
+ }
default: {
DefaultMetricsSystem.initialize("NameNode");
return new NameNode(conf);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
index ed6ff1a8a1c..deda30f1caa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
@@ -81,6 +81,7 @@ public class BootstrapStandby implements Tool, Configurable {
private boolean force = false;
private boolean interactive = true;
+ private boolean skipSharedEditsCheck = false;
// Exit/return codes.
static final int ERR_CODE_FAILED_CONNECT = 2;
@@ -117,6 +118,8 @@ public class BootstrapStandby implements Tool, Configurable {
force = true;
} else if ("-nonInteractive".equals(arg)) {
interactive = false;
+ } else if ("-skipSharedEditsCheck".equals(arg)) {
+ skipSharedEditsCheck = true;
} else {
printUsage();
throw new HadoopIllegalArgumentException(
@@ -127,7 +130,7 @@ public class BootstrapStandby implements Tool, Configurable {
private void printUsage() {
System.err.println("Usage: " + this.getClass().getSimpleName() +
- "[-force] [-nonInteractive]");
+ " [-force] [-nonInteractive] [-skipSharedEditsCheck]");
}
private NamenodeProtocol createNNProtocolProxy()
@@ -200,7 +203,7 @@ public class BootstrapStandby implements Tool, Configurable {
// Ensure that we have enough edits already in the shared directory to
// start up from the last checkpoint on the active.
- if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) {
+ if (!skipSharedEditsCheck && !checkLogsAvailableForRead(image, imageTxId, curTxId)) {
return ERR_CODE_LOGS_UNAVAILABLE;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index 0c5fed24395..e1fdfa341e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -24,6 +24,29 @@
Browsing HDFS
+
+
+